github.com/apache/arrow/go/v14@v14.0.2/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 "fmt" 23 "math" 24 "strconv" 25 "strings" 26 "testing" 27 28 "github.com/apache/arrow/go/v14/arrow" 29 "github.com/apache/arrow/go/v14/arrow/array" 30 "github.com/apache/arrow/go/v14/arrow/bitutil" 31 "github.com/apache/arrow/go/v14/arrow/decimal128" 32 "github.com/apache/arrow/go/v14/arrow/decimal256" 33 "github.com/apache/arrow/go/v14/arrow/ipc" 34 "github.com/apache/arrow/go/v14/arrow/memory" 35 "github.com/apache/arrow/go/v14/internal/types" 36 "github.com/apache/arrow/go/v14/internal/utils" 37 "github.com/apache/arrow/go/v14/parquet" 38 "github.com/apache/arrow/go/v14/parquet/compress" 39 "github.com/apache/arrow/go/v14/parquet/file" 40 "github.com/apache/arrow/go/v14/parquet/internal/encoding" 41 "github.com/apache/arrow/go/v14/parquet/internal/testutils" 42 "github.com/apache/arrow/go/v14/parquet/pqarrow" 43 "github.com/apache/arrow/go/v14/parquet/schema" 44 "github.com/google/uuid" 45 "github.com/stretchr/testify/assert" 46 "github.com/stretchr/testify/require" 47 "github.com/stretchr/testify/suite" 48 ) 49 50 func makeSimpleTable(values *arrow.Chunked, nullable bool) arrow.Table { 51 sc := arrow.NewSchema([]arrow.Field{{Name: "col", Type: values.DataType(), Nullable: nullable, 52 Metadata: arrow.NewMetadata([]string{"PARQUET:field_id"}, []string{"-1"})}}, nil) 53 column := arrow.NewColumn(sc.Field(0), values) 54 defer column.Release() 55 return array.NewTable(sc, []arrow.Column{*column}, -1) 56 } 57 58 func makeDateTimeTypesTable(mem memory.Allocator, expected bool, addFieldMeta bool) arrow.Table { 59 isValid := []bool{true, true, true, false, true, true} 60 61 // roundtrip without modification 62 f0 := arrow.Field{Name: "f0", Type: arrow.FixedWidthTypes.Date32, Nullable: true} 63 f1 := arrow.Field{Name: "f1", Type: arrow.FixedWidthTypes.Timestamp_ms, Nullable: true} 64 f2 := arrow.Field{Name: "f2", Type: arrow.FixedWidthTypes.Timestamp_us, Nullable: true} 65 f3 := arrow.Field{Name: "f3", Type: arrow.FixedWidthTypes.Timestamp_ns, Nullable: true} 66 f3X := arrow.Field{Name: "f3", Type: arrow.FixedWidthTypes.Timestamp_us, Nullable: true} 67 f4 := arrow.Field{Name: "f4", Type: arrow.FixedWidthTypes.Time32ms, Nullable: true} 68 f5 := arrow.Field{Name: "f5", Type: arrow.FixedWidthTypes.Time64us, Nullable: true} 69 f6 := arrow.Field{Name: "f6", Type: arrow.FixedWidthTypes.Time64ns, Nullable: true} 70 71 fieldList := []arrow.Field{f0, f1, f2} 72 if expected { 73 fieldList = append(fieldList, f3X) 74 } else { 75 fieldList = append(fieldList, f3) 76 } 77 fieldList = append(fieldList, f4, f5, f6) 78 79 if addFieldMeta { 80 for idx := range fieldList { 81 fieldList[idx].Metadata = arrow.NewMetadata([]string{"PARQUET:field_id"}, []string{strconv.Itoa(idx + 1)}) 82 } 83 } 84 arrsc := arrow.NewSchema(fieldList, nil) 85 86 d32Values := []arrow.Date32{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 87 ts64nsValues := []arrow.Timestamp{1489269000000, 1489270000000, 1489271000000, 1489272000000, 1489272000000, 1489273000000} 88 ts64usValues := []arrow.Timestamp{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 89 ts64msValues := []arrow.Timestamp{1489269, 1489270, 1489271, 1489272, 1489272, 1489273} 90 t32Values := []arrow.Time32{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 91 t64nsValues := []arrow.Time64{1489269000000, 1489270000000, 1489271000000, 1489272000000, 1489272000000, 1489273000000} 92 t64usValues := []arrow.Time64{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 93 94 builders := make([]array.Builder, 0, len(fieldList)) 95 for _, f := range fieldList { 96 bldr := array.NewBuilder(mem, f.Type) 97 defer bldr.Release() 98 builders = append(builders, bldr) 99 } 100 101 builders[0].(*array.Date32Builder).AppendValues(d32Values, isValid) 102 builders[1].(*array.TimestampBuilder).AppendValues(ts64msValues, isValid) 103 builders[2].(*array.TimestampBuilder).AppendValues(ts64usValues, isValid) 104 if expected { 105 builders[3].(*array.TimestampBuilder).AppendValues(ts64usValues, isValid) 106 } else { 107 builders[3].(*array.TimestampBuilder).AppendValues(ts64nsValues, isValid) 108 } 109 builders[4].(*array.Time32Builder).AppendValues(t32Values, isValid) 110 builders[5].(*array.Time64Builder).AppendValues(t64usValues, isValid) 111 builders[6].(*array.Time64Builder).AppendValues(t64nsValues, isValid) 112 113 cols := make([]arrow.Column, 0, len(fieldList)) 114 for idx, field := range fieldList { 115 arr := builders[idx].NewArray() 116 defer arr.Release() 117 118 chunked := arrow.NewChunked(field.Type, []arrow.Array{arr}) 119 defer chunked.Release() 120 col := arrow.NewColumn(field, chunked) 121 defer col.Release() 122 cols = append(cols, *col) 123 } 124 125 return array.NewTable(arrsc, cols, int64(len(isValid))) 126 } 127 128 func TestWriteArrowCols(t *testing.T) { 129 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 130 defer mem.AssertSize(t, 0) 131 132 tbl := makeDateTimeTypesTable(mem, false, false) 133 defer tbl.Release() 134 135 psc, err := pqarrow.ToParquet(tbl.Schema(), nil, pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 136 require.NoError(t, err) 137 138 manifest, err := pqarrow.NewSchemaManifest(psc, nil, nil) 139 require.NoError(t, err) 140 141 sink := encoding.NewBufferWriter(0, mem) 142 defer sink.Release() 143 writer := file.NewParquetWriter(sink, psc.Root(), file.WithWriterProps(parquet.NewWriterProperties(parquet.WithVersion(parquet.V2_4)))) 144 145 srgw := writer.AppendRowGroup() 146 ctx := pqarrow.NewArrowWriteContext(context.TODO(), nil) 147 148 for i := int64(0); i < tbl.NumCols(); i++ { 149 acw, err := pqarrow.NewArrowColumnWriter(tbl.Column(int(i)).Data(), 0, tbl.NumRows(), manifest, srgw, int(i)) 150 require.NoError(t, err) 151 require.NoError(t, acw.Write(ctx)) 152 } 153 require.NoError(t, srgw.Close()) 154 require.NoError(t, writer.Close()) 155 156 expected := makeDateTimeTypesTable(mem, true, false) 157 defer expected.Release() 158 159 reader, err := file.NewParquetReader(bytes.NewReader(sink.Bytes())) 160 require.NoError(t, err) 161 162 assert.EqualValues(t, expected.NumCols(), reader.MetaData().Schema.NumColumns()) 163 assert.EqualValues(t, expected.NumRows(), reader.NumRows()) 164 assert.EqualValues(t, 1, reader.NumRowGroups()) 165 166 rgr := reader.RowGroup(0) 167 168 for i := 0; i < int(expected.NumCols()); i++ { 169 var ( 170 total int64 171 read int 172 defLevelsOut = make([]int16, int(expected.NumRows())) 173 arr = expected.Column(i).Data().Chunk(0) 174 ) 175 switch expected.Schema().Field(i).Type.(arrow.FixedWidthDataType).BitWidth() { 176 case 32: 177 col, err := rgr.Column(i) 178 assert.NoError(t, err) 179 colReader := col.(*file.Int32ColumnChunkReader) 180 vals := make([]int32, int(expected.NumRows())) 181 total, read, err = colReader.ReadBatch(expected.NumRows(), vals, defLevelsOut, nil) 182 require.NoError(t, err) 183 184 nulls := 0 185 for j := 0; j < arr.Len(); j++ { 186 if arr.IsNull(j) { 187 nulls++ 188 continue 189 } 190 191 switch v := arr.(type) { 192 case *array.Date32: 193 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 194 case *array.Time32: 195 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 196 } 197 } 198 case 64: 199 col, err := rgr.Column(i) 200 assert.NoError(t, err) 201 colReader := col.(*file.Int64ColumnChunkReader) 202 vals := make([]int64, int(expected.NumRows())) 203 total, read, err = colReader.ReadBatch(expected.NumRows(), vals, defLevelsOut, nil) 204 require.NoError(t, err) 205 206 nulls := 0 207 for j := 0; j < arr.Len(); j++ { 208 if arr.IsNull(j) { 209 nulls++ 210 continue 211 } 212 213 switch v := arr.(type) { 214 case *array.Date64: 215 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 216 case *array.Time64: 217 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 218 case *array.Timestamp: 219 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 220 } 221 } 222 } 223 assert.EqualValues(t, expected.NumRows(), total) 224 assert.EqualValues(t, expected.NumRows()-1, read) 225 assert.Equal(t, []int16{1, 1, 1, 0, 1, 1}, defLevelsOut) 226 } 227 } 228 229 func TestWriteArrowInt96(t *testing.T) { 230 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 231 defer mem.AssertSize(t, 0) 232 233 tbl := makeDateTimeTypesTable(mem, false, false) 234 defer tbl.Release() 235 236 props := pqarrow.NewArrowWriterProperties(pqarrow.WithDeprecatedInt96Timestamps(true), pqarrow.WithAllocator(mem)) 237 238 psc, err := pqarrow.ToParquet(tbl.Schema(), nil, props) 239 require.NoError(t, err) 240 241 manifest, err := pqarrow.NewSchemaManifest(psc, nil, nil) 242 require.NoError(t, err) 243 244 sink := encoding.NewBufferWriter(0, mem) 245 defer sink.Release() 246 247 writer := file.NewParquetWriter(sink, psc.Root(), file.WithWriterProps(parquet.NewWriterProperties(parquet.WithAllocator(mem)))) 248 249 srgw := writer.AppendRowGroup() 250 ctx := pqarrow.NewArrowWriteContext(context.TODO(), &props) 251 252 for i := int64(0); i < tbl.NumCols(); i++ { 253 acw, err := pqarrow.NewArrowColumnWriter(tbl.Column(int(i)).Data(), 0, tbl.NumRows(), manifest, srgw, int(i)) 254 require.NoError(t, err) 255 require.NoError(t, acw.Write(ctx)) 256 } 257 require.NoError(t, srgw.Close()) 258 require.NoError(t, writer.Close()) 259 260 expected := makeDateTimeTypesTable(mem, false, false) 261 defer expected.Release() 262 263 reader, err := file.NewParquetReader(bytes.NewReader(sink.Bytes())) 264 require.NoError(t, err) 265 266 assert.EqualValues(t, expected.NumCols(), reader.MetaData().Schema.NumColumns()) 267 assert.EqualValues(t, expected.NumRows(), reader.NumRows()) 268 assert.EqualValues(t, 1, reader.NumRowGroups()) 269 270 rgr := reader.RowGroup(0) 271 tsRdr, err := rgr.Column(3) 272 assert.NoError(t, err) 273 assert.Equal(t, parquet.Types.Int96, tsRdr.Type()) 274 275 rdr := tsRdr.(*file.Int96ColumnChunkReader) 276 vals := make([]parquet.Int96, expected.NumRows()) 277 defLevels := make([]int16, int(expected.NumRows())) 278 279 total, read, _ := rdr.ReadBatch(expected.NumRows(), vals, defLevels, nil) 280 assert.EqualValues(t, expected.NumRows(), total) 281 assert.EqualValues(t, expected.NumRows()-1, read) 282 assert.Equal(t, []int16{1, 1, 1, 0, 1, 1}, defLevels) 283 284 data := expected.Column(3).Data().Chunk(0).(*array.Timestamp) 285 assert.EqualValues(t, data.Value(0), vals[0].ToTime().UnixNano()) 286 assert.EqualValues(t, data.Value(1), vals[1].ToTime().UnixNano()) 287 assert.EqualValues(t, data.Value(2), vals[2].ToTime().UnixNano()) 288 assert.EqualValues(t, data.Value(4), vals[3].ToTime().UnixNano()) 289 assert.EqualValues(t, data.Value(5), vals[4].ToTime().UnixNano()) 290 } 291 292 func writeTableToBuffer(t *testing.T, mem memory.Allocator, tbl arrow.Table, rowGroupSize int64, props pqarrow.ArrowWriterProperties) *memory.Buffer { 293 sink := encoding.NewBufferWriter(0, mem) 294 defer sink.Release() 295 wrprops := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1_0)) 296 psc, err := pqarrow.ToParquet(tbl.Schema(), wrprops, props) 297 require.NoError(t, err) 298 299 manifest, err := pqarrow.NewSchemaManifest(psc, nil, nil) 300 require.NoError(t, err) 301 302 writer := file.NewParquetWriter(sink, psc.Root(), file.WithWriterProps(wrprops)) 303 ctx := pqarrow.NewArrowWriteContext(context.TODO(), &props) 304 305 offset := int64(0) 306 for offset < tbl.NumRows() { 307 sz := utils.Min(rowGroupSize, tbl.NumRows()-offset) 308 srgw := writer.AppendRowGroup() 309 for i := 0; i < int(tbl.NumCols()); i++ { 310 col := tbl.Column(i) 311 acw, err := pqarrow.NewArrowColumnWriter(col.Data(), offset, sz, manifest, srgw, i) 312 require.NoError(t, err) 313 require.NoError(t, acw.Write(ctx)) 314 } 315 srgw.Close() 316 offset += sz 317 } 318 writer.Close() 319 320 return sink.Finish() 321 } 322 323 func simpleRoundTrip(t *testing.T, tbl arrow.Table, rowGroupSize int64) { 324 t.Helper() 325 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 326 defer mem.AssertSize(t, 0) 327 328 buf := writeTableToBuffer(t, mem, tbl, rowGroupSize, pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 329 defer buf.Release() 330 331 rdr, err := file.NewParquetReader(bytes.NewReader(buf.Bytes())) 332 require.NoError(t, err) 333 334 ardr, err := pqarrow.NewFileReader(rdr, pqarrow.ArrowReadProperties{}, mem) 335 require.NoError(t, err) 336 337 for i := 0; i < int(tbl.NumCols()); i++ { 338 crdr, err := ardr.GetColumn(context.TODO(), i) 339 require.NoError(t, err) 340 341 chunked, err := crdr.NextBatch(tbl.NumRows()) 342 require.NoError(t, err) 343 defer chunked.Release() 344 345 require.EqualValues(t, tbl.NumRows(), chunked.Len()) 346 347 chunkList := tbl.Column(i).Data().Chunks() 348 offset := int64(0) 349 for _, chnk := range chunkList { 350 slc := array.NewChunkedSlice(chunked, offset, offset+int64(chnk.Len())) 351 defer slc.Release() 352 353 assert.EqualValues(t, chnk.Len(), slc.Len()) 354 if len(slc.Chunks()) == 1 { 355 offset += int64(chnk.Len()) 356 assert.True(t, array.Equal(chnk, slc.Chunk(0))) 357 } 358 } 359 crdr.Release() 360 } 361 } 362 363 func TestWriteKeyValueMetadata(t *testing.T) { 364 kv := map[string]string{ 365 "key1": "value1", 366 "key2": "value2", 367 "key3": "value3", 368 } 369 370 sc := arrow.NewSchema([]arrow.Field{ 371 {Name: "int32", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, 372 }, nil) 373 bldr := array.NewRecordBuilder(memory.DefaultAllocator, sc) 374 defer bldr.Release() 375 for _, b := range bldr.Fields() { 376 b.AppendNull() 377 } 378 379 rec := bldr.NewRecord() 380 defer rec.Release() 381 382 props := parquet.NewWriterProperties( 383 parquet.WithVersion(parquet.V1_0), 384 ) 385 var buf bytes.Buffer 386 fw, err := pqarrow.NewFileWriter(sc, &buf, props, pqarrow.DefaultWriterProps()) 387 require.NoError(t, err) 388 err = fw.Write(rec) 389 require.NoError(t, err) 390 391 for key, value := range kv { 392 require.NoError(t, fw.AppendKeyValueMetadata(key, value)) 393 } 394 395 err = fw.Close() 396 require.NoError(t, err) 397 398 reader, err := file.NewParquetReader(bytes.NewReader(buf.Bytes())) 399 require.NoError(t, err) 400 401 for key, value := range kv { 402 got := reader.MetaData().KeyValueMetadata().FindValue(key) 403 require.NotNil(t, got) 404 assert.Equal(t, value, *got) 405 } 406 } 407 408 func TestWriteEmptyLists(t *testing.T) { 409 sc := arrow.NewSchema([]arrow.Field{ 410 {Name: "f1", Type: arrow.ListOf(arrow.FixedWidthTypes.Date32)}, 411 {Name: "f2", Type: arrow.ListOf(arrow.FixedWidthTypes.Date64)}, 412 {Name: "f3", Type: arrow.ListOf(arrow.FixedWidthTypes.Timestamp_us)}, 413 {Name: "f4", Type: arrow.ListOf(arrow.FixedWidthTypes.Timestamp_ms)}, 414 {Name: "f5", Type: arrow.ListOf(arrow.FixedWidthTypes.Time32ms)}, 415 {Name: "f6", Type: arrow.ListOf(arrow.FixedWidthTypes.Time64ns)}, 416 {Name: "f7", Type: arrow.ListOf(arrow.FixedWidthTypes.Time64us)}, 417 }, nil) 418 bldr := array.NewRecordBuilder(memory.DefaultAllocator, sc) 419 defer bldr.Release() 420 for _, b := range bldr.Fields() { 421 b.AppendNull() 422 } 423 424 rec := bldr.NewRecord() 425 defer rec.Release() 426 427 props := parquet.NewWriterProperties( 428 parquet.WithVersion(parquet.V1_0), 429 ) 430 arrprops := pqarrow.DefaultWriterProps() 431 var buf bytes.Buffer 432 fw, err := pqarrow.NewFileWriter(sc, &buf, props, arrprops) 433 require.NoError(t, err) 434 err = fw.Write(rec) 435 require.NoError(t, err) 436 err = fw.Close() 437 require.NoError(t, err) 438 } 439 440 func TestArrowReadWriteTableChunkedCols(t *testing.T) { 441 chunkSizes := []int{2, 4, 10, 2} 442 const totalLen = int64(18) 443 444 rng := testutils.NewRandomArrayGenerator(0) 445 446 arr := rng.Int32(totalLen, 0, math.MaxInt32/2, 0.9) 447 defer arr.Release() 448 449 offset := int64(0) 450 chunks := make([]arrow.Array, 0) 451 for _, chnksize := range chunkSizes { 452 chk := array.NewSlice(arr, offset, offset+int64(chnksize)) 453 defer chk.Release() 454 defer chk.Release() // for NewChunked below 455 chunks = append(chunks, chk) 456 } 457 458 sc := arrow.NewSchema([]arrow.Field{{Name: "field", Type: arr.DataType(), Nullable: true}}, nil) 459 460 chk := arrow.NewChunked(arr.DataType(), chunks) 461 defer chk.Release() 462 463 tbl := array.NewTable(sc, []arrow.Column{*arrow.NewColumn(sc.Field(0), chk)}, -1) 464 defer tbl.Release() 465 466 simpleRoundTrip(t, tbl, 2) 467 simpleRoundTrip(t, tbl, 10) 468 } 469 470 // set this up for checking our expected results so we can test the functions 471 // that generate them which we export 472 func getLogicalType(typ arrow.DataType) schema.LogicalType { 473 switch typ.ID() { 474 case arrow.DICTIONARY: 475 return getLogicalType(typ.(*arrow.DictionaryType).ValueType) 476 case arrow.INT8: 477 return schema.NewIntLogicalType(8, true) 478 case arrow.UINT8: 479 return schema.NewIntLogicalType(8, false) 480 case arrow.INT16: 481 return schema.NewIntLogicalType(16, true) 482 case arrow.UINT16: 483 return schema.NewIntLogicalType(16, false) 484 case arrow.INT32: 485 return schema.NewIntLogicalType(32, true) 486 case arrow.UINT32: 487 return schema.NewIntLogicalType(32, false) 488 case arrow.INT64: 489 return schema.NewIntLogicalType(64, true) 490 case arrow.UINT64: 491 return schema.NewIntLogicalType(64, false) 492 case arrow.STRING, arrow.LARGE_STRING: 493 return schema.StringLogicalType{} 494 case arrow.DATE32: 495 return schema.DateLogicalType{} 496 case arrow.DATE64: 497 return schema.DateLogicalType{} 498 case arrow.TIMESTAMP: 499 ts := typ.(*arrow.TimestampType) 500 adjustedUTC := len(ts.TimeZone) == 0 501 switch ts.Unit { 502 case arrow.Microsecond: 503 return schema.NewTimestampLogicalType(adjustedUTC, schema.TimeUnitMicros) 504 case arrow.Millisecond: 505 return schema.NewTimestampLogicalType(adjustedUTC, schema.TimeUnitMillis) 506 case arrow.Nanosecond: 507 return schema.NewTimestampLogicalType(adjustedUTC, schema.TimeUnitNanos) 508 default: 509 panic("only milli, micro and nano units supported for arrow timestamp") 510 } 511 case arrow.TIME32: 512 return schema.NewTimeLogicalType(false, schema.TimeUnitMillis) 513 case arrow.TIME64: 514 ts := typ.(*arrow.Time64Type) 515 switch ts.Unit { 516 case arrow.Microsecond: 517 return schema.NewTimeLogicalType(false, schema.TimeUnitMicros) 518 case arrow.Nanosecond: 519 return schema.NewTimeLogicalType(false, schema.TimeUnitNanos) 520 default: 521 panic("only micro and nano seconds are supported for arrow TIME64") 522 } 523 case arrow.DECIMAL, arrow.DECIMAL256: 524 dec := typ.(arrow.DecimalType) 525 return schema.NewDecimalLogicalType(dec.GetPrecision(), dec.GetScale()) 526 } 527 return schema.NoLogicalType{} 528 } 529 530 func getPhysicalType(typ arrow.DataType) parquet.Type { 531 switch typ.ID() { 532 case arrow.DICTIONARY: 533 return getPhysicalType(typ.(*arrow.DictionaryType).ValueType) 534 case arrow.BOOL: 535 return parquet.Types.Boolean 536 case arrow.UINT8, arrow.INT8, arrow.UINT16, arrow.INT16, arrow.UINT32, arrow.INT32: 537 return parquet.Types.Int32 538 case arrow.INT64, arrow.UINT64: 539 return parquet.Types.Int64 540 case arrow.FLOAT32: 541 return parquet.Types.Float 542 case arrow.FLOAT64: 543 return parquet.Types.Double 544 case arrow.BINARY, arrow.LARGE_BINARY, arrow.STRING, arrow.LARGE_STRING: 545 return parquet.Types.ByteArray 546 case arrow.FIXED_SIZE_BINARY, arrow.DECIMAL: 547 return parquet.Types.FixedLenByteArray 548 case arrow.DATE32: 549 return parquet.Types.Int32 550 case arrow.DATE64: 551 // convert to date32 internally 552 return parquet.Types.Int32 553 case arrow.TIME32: 554 return parquet.Types.Int32 555 case arrow.TIME64, arrow.TIMESTAMP: 556 return parquet.Types.Int64 557 default: 558 return parquet.Types.Int32 559 } 560 } 561 562 const ( 563 boolTestValue = true 564 uint8TestVal = uint8(64) 565 int8TestVal = int8(-64) 566 uint16TestVal = uint16(1024) 567 int16TestVal = int16(-1024) 568 uint32TestVal = uint32(1024) 569 int32TestVal = int32(-1024) 570 uint64TestVal = uint64(1024) 571 int64TestVal = int64(-1024) 572 tsTestValue = arrow.Timestamp(14695634030000) 573 date32TestVal = arrow.Date32(170000) 574 floatTestVal = float32(2.1) 575 doubleTestVal = float64(4.2) 576 strTestVal = "Test" 577 578 smallSize = 100 579 ) 580 581 type ParquetIOTestSuite struct { 582 suite.Suite 583 } 584 585 func (ps *ParquetIOTestSuite) SetupTest() { 586 ps.NoError(arrow.RegisterExtensionType(types.NewUUIDType())) 587 } 588 589 func (ps *ParquetIOTestSuite) TearDownTest() { 590 if arrow.GetExtensionType("uuid") != nil { 591 ps.NoError(arrow.UnregisterExtensionType("uuid")) 592 } 593 } 594 595 func (ps *ParquetIOTestSuite) makeSimpleSchema(typ arrow.DataType, rep parquet.Repetition) *schema.GroupNode { 596 byteWidth := int32(-1) 597 598 switch typ := typ.(type) { 599 case *arrow.FixedSizeBinaryType: 600 byteWidth = int32(typ.ByteWidth) 601 case arrow.DecimalType: 602 byteWidth = pqarrow.DecimalSize(typ.GetPrecision()) 603 case *arrow.DictionaryType: 604 valuesType := typ.ValueType 605 switch dt := valuesType.(type) { 606 case *arrow.FixedSizeBinaryType: 607 byteWidth = int32(dt.ByteWidth) 608 case arrow.DecimalType: 609 byteWidth = pqarrow.DecimalSize(dt.GetPrecision()) 610 } 611 } 612 613 pnode, _ := schema.NewPrimitiveNodeLogical("column1", rep, getLogicalType(typ), getPhysicalType(typ), int(byteWidth), -1) 614 return schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{pnode}, -1)) 615 } 616 617 func (ps *ParquetIOTestSuite) makePrimitiveTestCol(mem memory.Allocator, size int, typ arrow.DataType) arrow.Array { 618 switch typ.ID() { 619 case arrow.BOOL: 620 bldr := array.NewBooleanBuilder(mem) 621 defer bldr.Release() 622 for i := 0; i < size; i++ { 623 bldr.Append(boolTestValue) 624 } 625 return bldr.NewArray() 626 case arrow.INT8: 627 bldr := array.NewInt8Builder(mem) 628 defer bldr.Release() 629 for i := 0; i < size; i++ { 630 bldr.Append(int8TestVal) 631 } 632 return bldr.NewArray() 633 case arrow.UINT8: 634 bldr := array.NewUint8Builder(mem) 635 defer bldr.Release() 636 for i := 0; i < size; i++ { 637 bldr.Append(uint8TestVal) 638 } 639 return bldr.NewArray() 640 case arrow.INT16: 641 bldr := array.NewInt16Builder(mem) 642 defer bldr.Release() 643 for i := 0; i < size; i++ { 644 bldr.Append(int16TestVal) 645 } 646 return bldr.NewArray() 647 case arrow.UINT16: 648 bldr := array.NewUint16Builder(mem) 649 defer bldr.Release() 650 for i := 0; i < size; i++ { 651 bldr.Append(uint16TestVal) 652 } 653 return bldr.NewArray() 654 case arrow.INT32: 655 bldr := array.NewInt32Builder(mem) 656 defer bldr.Release() 657 for i := 0; i < size; i++ { 658 bldr.Append(int32TestVal) 659 } 660 return bldr.NewArray() 661 case arrow.UINT32: 662 bldr := array.NewUint32Builder(mem) 663 defer bldr.Release() 664 for i := 0; i < size; i++ { 665 bldr.Append(uint32TestVal) 666 } 667 return bldr.NewArray() 668 case arrow.INT64: 669 bldr := array.NewInt64Builder(mem) 670 defer bldr.Release() 671 for i := 0; i < size; i++ { 672 bldr.Append(int64TestVal) 673 } 674 return bldr.NewArray() 675 case arrow.UINT64: 676 bldr := array.NewUint64Builder(mem) 677 defer bldr.Release() 678 for i := 0; i < size; i++ { 679 bldr.Append(uint64TestVal) 680 } 681 return bldr.NewArray() 682 case arrow.FLOAT32: 683 bldr := array.NewFloat32Builder(mem) 684 defer bldr.Release() 685 for i := 0; i < size; i++ { 686 bldr.Append(floatTestVal) 687 } 688 return bldr.NewArray() 689 case arrow.FLOAT64: 690 bldr := array.NewFloat64Builder(mem) 691 defer bldr.Release() 692 for i := 0; i < size; i++ { 693 bldr.Append(doubleTestVal) 694 } 695 return bldr.NewArray() 696 } 697 return nil 698 } 699 700 func (ps *ParquetIOTestSuite) makeTestFile(mem memory.Allocator, typ arrow.DataType, arr arrow.Array, numChunks int) []byte { 701 sc := ps.makeSimpleSchema(typ, parquet.Repetitions.Required) 702 sink := encoding.NewBufferWriter(0, mem) 703 defer sink.Release() 704 writer := file.NewParquetWriter(sink, sc, file.WithWriterProps(parquet.NewWriterProperties(parquet.WithAllocator(mem)))) 705 706 props := pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem)) 707 ctx := pqarrow.NewArrowWriteContext(context.TODO(), &props) 708 rowGroupSize := arr.Len() / numChunks 709 710 for i := 0; i < numChunks; i++ { 711 rgw := writer.AppendRowGroup() 712 cw, err := rgw.NextColumn() 713 ps.NoError(err) 714 715 start := i * rowGroupSize 716 slc := array.NewSlice(arr, int64(start), int64(start+rowGroupSize)) 717 defer slc.Release() 718 ps.NoError(pqarrow.WriteArrowToColumn(ctx, cw, slc, nil, nil, false)) 719 ps.NoError(cw.Close()) 720 ps.NoError(rgw.Close()) 721 } 722 ps.NoError(writer.Close()) 723 buf := sink.Finish() 724 defer buf.Release() 725 return buf.Bytes() 726 } 727 728 func (ps *ParquetIOTestSuite) createReader(mem memory.Allocator, data []byte) *pqarrow.FileReader { 729 rdr, err := file.NewParquetReader(bytes.NewReader(data), file.WithReadProps(parquet.NewReaderProperties(mem))) 730 ps.NoError(err) 731 732 reader, err := pqarrow.NewFileReader(rdr, pqarrow.ArrowReadProperties{}, mem) 733 ps.NoError(err) 734 return reader 735 } 736 737 func (ps *ParquetIOTestSuite) readTable(rdr *pqarrow.FileReader) arrow.Table { 738 tbl, err := rdr.ReadTable(context.TODO()) 739 ps.NoError(err) 740 ps.NotNil(tbl) 741 return tbl 742 } 743 744 func (ps *ParquetIOTestSuite) checkSingleColumnRequiredTableRead(mem memory.Allocator, typ arrow.DataType, numChunks int) { 745 values := ps.makePrimitiveTestCol(mem, smallSize, typ) 746 defer values.Release() 747 748 data := ps.makeTestFile(mem, typ, values, numChunks) 749 reader := ps.createReader(mem, data) 750 751 tbl := ps.readTable(reader) 752 defer tbl.Release() 753 754 ps.EqualValues(1, tbl.NumCols()) 755 ps.EqualValues(smallSize, tbl.NumRows()) 756 757 chunked := tbl.Column(0).Data() 758 ps.Len(chunked.Chunks(), 1) 759 ps.True(array.Equal(values, chunked.Chunk(0))) 760 } 761 762 func (ps *ParquetIOTestSuite) checkSingleColumnRead(mem memory.Allocator, typ arrow.DataType, numChunks int) { 763 values := ps.makePrimitiveTestCol(mem, smallSize, typ) 764 defer values.Release() 765 766 data := ps.makeTestFile(mem, typ, values, numChunks) 767 reader := ps.createReader(mem, data) 768 769 cr, err := reader.GetColumn(context.TODO(), 0) 770 ps.NoError(err) 771 defer cr.Release() 772 773 chunked, err := cr.NextBatch(smallSize) 774 ps.NoError(err) 775 defer chunked.Release() 776 777 ps.Len(chunked.Chunks(), 1) 778 ps.True(array.Equal(values, chunked.Chunk(0))) 779 } 780 781 func (ps *ParquetIOTestSuite) TestDateTimeTypesReadWriteTable() { 782 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 783 defer mem.AssertSize(ps.T(), 0) 784 785 toWrite := makeDateTimeTypesTable(mem, false, true) 786 defer toWrite.Release() 787 buf := writeTableToBuffer(ps.T(), mem, toWrite, toWrite.NumRows(), pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 788 defer buf.Release() 789 790 reader := ps.createReader(mem, buf.Bytes()) 791 tbl := ps.readTable(reader) 792 defer tbl.Release() 793 794 expected := makeDateTimeTypesTable(mem, true, true) 795 defer expected.Release() 796 797 ps.Equal(expected.NumCols(), tbl.NumCols()) 798 ps.Equal(expected.NumRows(), tbl.NumRows()) 799 ps.Truef(expected.Schema().Equal(tbl.Schema()), "expected schema: %s\ngot schema: %s", expected.Schema(), tbl.Schema()) 800 801 for i := 0; i < int(expected.NumCols()); i++ { 802 exChunk := expected.Column(i).Data() 803 tblChunk := tbl.Column(i).Data() 804 805 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 806 ps.Truef(array.Equal(exChunk.Chunk(0), tblChunk.Chunk(0)), "expected %s\ngot %s", exChunk.Chunk(0), tblChunk.Chunk(0)) 807 } 808 } 809 810 func (ps *ParquetIOTestSuite) TestDateTimeTypesWithInt96ReadWriteTable() { 811 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 812 defer mem.AssertSize(ps.T(), 0) 813 814 expected := makeDateTimeTypesTable(mem, false, true) 815 defer expected.Release() 816 buf := writeTableToBuffer(ps.T(), mem, expected, expected.NumRows(), pqarrow.NewArrowWriterProperties(pqarrow.WithDeprecatedInt96Timestamps(true))) 817 defer buf.Release() 818 819 reader := ps.createReader(mem, buf.Bytes()) 820 tbl := ps.readTable(reader) 821 defer tbl.Release() 822 823 ps.Equal(expected.NumCols(), tbl.NumCols()) 824 ps.Equal(expected.NumRows(), tbl.NumRows()) 825 ps.Truef(expected.Schema().Equal(tbl.Schema()), "expected schema: %s\ngot schema: %s", expected.Schema(), tbl.Schema()) 826 827 for i := 0; i < int(expected.NumCols()); i++ { 828 exChunk := expected.Column(i).Data() 829 tblChunk := tbl.Column(i).Data() 830 831 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 832 ps.Truef(array.Equal(exChunk.Chunk(0), tblChunk.Chunk(0)), "expected %s\ngot %s", exChunk.Chunk(0), tblChunk.Chunk(0)) 833 } 834 } 835 836 func (ps *ParquetIOTestSuite) TestLargeBinaryReadWriteTable() { 837 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 838 defer mem.AssertSize(ps.T(), 0) 839 840 // While we may write using LargeString, when we read, we get an array.String back out. 841 // So we're building a normal array.String to use with array.Equal 842 lsBldr := array.NewLargeStringBuilder(mem) 843 defer lsBldr.Release() 844 lbBldr := array.NewBinaryBuilder(mem, arrow.BinaryTypes.LargeBinary) 845 defer lbBldr.Release() 846 847 for i := 0; i < smallSize; i++ { 848 s := strconv.FormatInt(int64(i), 10) 849 lsBldr.Append(s) 850 lbBldr.Append([]byte(s)) 851 } 852 853 lsValues := lsBldr.NewArray() 854 defer lsValues.Release() 855 lbValues := lbBldr.NewArray() 856 defer lbValues.Release() 857 858 lsField := arrow.Field{Name: "large_string", Type: arrow.BinaryTypes.LargeString, Nullable: true} 859 lbField := arrow.Field{Name: "large_binary", Type: arrow.BinaryTypes.LargeBinary, Nullable: true} 860 expected := array.NewTable( 861 arrow.NewSchema([]arrow.Field{lsField, lbField}, nil), 862 []arrow.Column{ 863 *arrow.NewColumn(lsField, arrow.NewChunked(lsField.Type, []arrow.Array{lsValues})), 864 *arrow.NewColumn(lbField, arrow.NewChunked(lbField.Type, []arrow.Array{lbValues})), 865 }, 866 -1, 867 ) 868 defer lsValues.Release() // NewChunked 869 defer lbValues.Release() // NewChunked 870 defer expected.Release() 871 ps.roundTripTable(mem, expected, true) 872 } 873 874 func (ps *ParquetIOTestSuite) TestReadSingleColumnFile() { 875 types := []arrow.DataType{ 876 arrow.FixedWidthTypes.Boolean, 877 arrow.PrimitiveTypes.Uint8, 878 arrow.PrimitiveTypes.Int8, 879 arrow.PrimitiveTypes.Uint16, 880 arrow.PrimitiveTypes.Int16, 881 arrow.PrimitiveTypes.Uint32, 882 arrow.PrimitiveTypes.Int32, 883 arrow.PrimitiveTypes.Uint64, 884 arrow.PrimitiveTypes.Int64, 885 arrow.PrimitiveTypes.Float32, 886 arrow.PrimitiveTypes.Float64, 887 } 888 889 nchunks := []int{1, 4} 890 891 for _, n := range nchunks { 892 for _, dt := range types { 893 ps.Run(fmt.Sprintf("%s %d chunks", dt.Name(), n), func() { 894 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 895 defer mem.AssertSize(ps.T(), 0) 896 ps.checkSingleColumnRead(mem, dt, n) 897 }) 898 } 899 } 900 } 901 902 func (ps *ParquetIOTestSuite) TestSingleColumnRequiredRead() { 903 types := []arrow.DataType{ 904 arrow.FixedWidthTypes.Boolean, 905 arrow.PrimitiveTypes.Uint8, 906 arrow.PrimitiveTypes.Int8, 907 arrow.PrimitiveTypes.Uint16, 908 arrow.PrimitiveTypes.Int16, 909 arrow.PrimitiveTypes.Uint32, 910 arrow.PrimitiveTypes.Int32, 911 arrow.PrimitiveTypes.Uint64, 912 arrow.PrimitiveTypes.Int64, 913 arrow.PrimitiveTypes.Float32, 914 arrow.PrimitiveTypes.Float64, 915 } 916 917 nchunks := []int{1, 4} 918 919 for _, n := range nchunks { 920 for _, dt := range types { 921 ps.Run(fmt.Sprintf("%s %d chunks", dt.Name(), n), func() { 922 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 923 defer mem.AssertSize(ps.T(), 0) 924 925 ps.checkSingleColumnRequiredTableRead(mem, dt, n) 926 }) 927 } 928 } 929 } 930 931 func (ps *ParquetIOTestSuite) TestReadDecimals() { 932 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 933 defer mem.AssertSize(ps.T(), 0) 934 935 bigEndian := []parquet.ByteArray{ 936 // 123456 937 []byte{1, 226, 64}, 938 // 987654 939 []byte{15, 18, 6}, 940 // -123456 941 []byte{255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 254, 29, 192}, 942 } 943 944 bldr := array.NewDecimal128Builder(mem, &arrow.Decimal128Type{Precision: 6, Scale: 3}) 945 defer bldr.Release() 946 947 bldr.Append(decimal128.FromU64(123456)) 948 bldr.Append(decimal128.FromU64(987654)) 949 bldr.Append(decimal128.FromI64(-123456)) 950 951 expected := bldr.NewDecimal128Array() 952 defer expected.Release() 953 954 sc := schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{ 955 schema.Must(schema.NewPrimitiveNodeLogical("decimals", parquet.Repetitions.Required, schema.NewDecimalLogicalType(6, 3), parquet.Types.ByteArray, -1, -1)), 956 }, -1)) 957 958 sink := encoding.NewBufferWriter(0, mem) 959 defer sink.Release() 960 writer := file.NewParquetWriter(sink, sc) 961 962 rgw := writer.AppendRowGroup() 963 cw, _ := rgw.NextColumn() 964 cw.(*file.ByteArrayColumnChunkWriter).WriteBatch(bigEndian, nil, nil) 965 cw.Close() 966 rgw.Close() 967 writer.Close() 968 969 rdr := ps.createReader(mem, sink.Bytes()) 970 cr, err := rdr.GetColumn(context.TODO(), 0) 971 ps.NoError(err) 972 973 chunked, err := cr.NextBatch(smallSize) 974 ps.NoError(err) 975 defer chunked.Release() 976 977 ps.Len(chunked.Chunks(), 1) 978 ps.True(array.Equal(expected, chunked.Chunk(0))) 979 } 980 981 func (ps *ParquetIOTestSuite) TestReadDecimal256() { 982 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 983 defer mem.AssertSize(ps.T(), 0) 984 985 bigEndian := []parquet.ByteArray{ 986 // 123456 987 []byte{1, 226, 64}, 988 // 987654 989 []byte{15, 18, 6}, 990 // -123456 991 []byte{255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 254, 29, 192}, 992 } 993 994 bldr := array.NewDecimal256Builder(mem, &arrow.Decimal256Type{Precision: 40, Scale: 3}) 995 defer bldr.Release() 996 997 bldr.Append(decimal256.FromU64(123456)) 998 bldr.Append(decimal256.FromU64(987654)) 999 bldr.Append(decimal256.FromI64(-123456)) 1000 1001 expected := bldr.NewDecimal256Array() 1002 defer expected.Release() 1003 1004 sc := schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{ 1005 schema.Must(schema.NewPrimitiveNodeLogical("decimals", parquet.Repetitions.Required, schema.NewDecimalLogicalType(40, 3), parquet.Types.ByteArray, -1, -1)), 1006 }, -1)) 1007 1008 sink := encoding.NewBufferWriter(0, mem) 1009 defer sink.Release() 1010 writer := file.NewParquetWriter(sink, sc) 1011 1012 rgw := writer.AppendRowGroup() 1013 cw, _ := rgw.NextColumn() 1014 cw.(*file.ByteArrayColumnChunkWriter).WriteBatch(bigEndian, nil, nil) 1015 cw.Close() 1016 rgw.Close() 1017 writer.Close() 1018 1019 rdr := ps.createReader(mem, sink.Bytes()) 1020 cr, err := rdr.GetColumn(context.TODO(), 0) 1021 ps.NoError(err) 1022 1023 chunked, err := cr.NextBatch(smallSize) 1024 ps.NoError(err) 1025 defer chunked.Release() 1026 1027 ps.Len(chunked.Chunks(), 1) 1028 ps.Truef(array.Equal(expected, chunked.Chunk(0)), "expected: %s\ngot: %s", expected, chunked.Chunk(0)) 1029 } 1030 1031 func (ps *ParquetIOTestSuite) TestReadNestedStruct() { 1032 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1033 defer mem.AssertSize(ps.T(), 0) 1034 1035 dt := arrow.StructOf(arrow.Field{ 1036 Name: "nested", 1037 Type: arrow.StructOf( 1038 arrow.Field{Name: "bool", Type: arrow.FixedWidthTypes.Boolean}, 1039 arrow.Field{Name: "int32", Type: arrow.PrimitiveTypes.Int32}, 1040 arrow.Field{Name: "int64", Type: arrow.PrimitiveTypes.Int64}, 1041 ), 1042 }) 1043 field := arrow.Field{Name: "struct", Type: dt, Nullable: true} 1044 1045 builder := array.NewStructBuilder(mem, dt) 1046 defer builder.Release() 1047 nested := builder.FieldBuilder(0).(*array.StructBuilder) 1048 1049 builder.Append(true) 1050 nested.Append(true) 1051 nested.FieldBuilder(0).(*array.BooleanBuilder).Append(true) 1052 nested.FieldBuilder(1).(*array.Int32Builder).Append(int32(-1)) 1053 nested.FieldBuilder(2).(*array.Int64Builder).Append(int64(-2)) 1054 builder.AppendNull() 1055 1056 arr := builder.NewStructArray() 1057 defer arr.Release() 1058 1059 expected := array.NewTable( 1060 arrow.NewSchema([]arrow.Field{field}, nil), 1061 []arrow.Column{*arrow.NewColumn(field, arrow.NewChunked(dt, []arrow.Array{arr}))}, 1062 -1, 1063 ) 1064 defer arr.Release() // NewChunked 1065 defer expected.Release() 1066 ps.roundTripTable(mem, expected, true) 1067 } 1068 1069 func (ps *ParquetIOTestSuite) writeColumn(mem memory.Allocator, sc *schema.GroupNode, values arrow.Array) []byte { 1070 var buf bytes.Buffer 1071 arrsc, err := pqarrow.FromParquet(schema.NewSchema(sc), nil, nil) 1072 ps.NoError(err) 1073 1074 writer, err := pqarrow.NewFileWriter(arrsc, &buf, parquet.NewWriterProperties(parquet.WithDictionaryDefault(false)), pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 1075 ps.NoError(err) 1076 1077 writer.NewRowGroup() 1078 ps.NoError(writer.WriteColumnData(values)) 1079 //defer values.Release() 1080 ps.NoError(writer.Close()) 1081 ps.NoError(writer.Close()) 1082 1083 return buf.Bytes() 1084 } 1085 1086 func (ps *ParquetIOTestSuite) readAndCheckSingleColumnFile(mem memory.Allocator, data []byte, values arrow.Array) { 1087 reader := ps.createReader(mem, data) 1088 cr, err := reader.GetColumn(context.TODO(), 0) 1089 ps.NoError(err) 1090 ps.NotNil(cr) 1091 defer cr.Release() 1092 1093 chunked, err := cr.NextBatch(smallSize) 1094 ps.NoError(err) 1095 defer chunked.Release() 1096 1097 ps.Len(chunked.Chunks(), 1) 1098 ps.NotNil(chunked.Chunk(0)) 1099 1100 ps.True(array.Equal(values, chunked.Chunk(0))) 1101 } 1102 1103 var fullTypeList = []arrow.DataType{ 1104 arrow.FixedWidthTypes.Boolean, 1105 arrow.PrimitiveTypes.Uint8, 1106 arrow.PrimitiveTypes.Int8, 1107 arrow.PrimitiveTypes.Uint16, 1108 arrow.PrimitiveTypes.Int16, 1109 arrow.PrimitiveTypes.Uint32, 1110 arrow.PrimitiveTypes.Int32, 1111 arrow.PrimitiveTypes.Uint64, 1112 arrow.PrimitiveTypes.Int64, 1113 arrow.FixedWidthTypes.Date32, 1114 arrow.PrimitiveTypes.Float32, 1115 arrow.PrimitiveTypes.Float64, 1116 arrow.BinaryTypes.String, 1117 arrow.BinaryTypes.Binary, 1118 &arrow.FixedSizeBinaryType{ByteWidth: 10}, 1119 &arrow.Decimal128Type{Precision: 1, Scale: 0}, 1120 &arrow.Decimal128Type{Precision: 5, Scale: 4}, 1121 &arrow.Decimal128Type{Precision: 10, Scale: 9}, 1122 &arrow.Decimal128Type{Precision: 19, Scale: 18}, 1123 &arrow.Decimal128Type{Precision: 23, Scale: 22}, 1124 &arrow.Decimal128Type{Precision: 27, Scale: 26}, 1125 &arrow.Decimal128Type{Precision: 38, Scale: 37}, 1126 } 1127 1128 func (ps *ParquetIOTestSuite) TestSingleColumnRequiredWrite() { 1129 for _, dt := range fullTypeList { 1130 ps.Run(dt.Name(), func() { 1131 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1132 defer mem.AssertSize(ps.T(), 0) 1133 1134 values := testutils.RandomNonNull(mem, dt, smallSize) 1135 defer values.Release() 1136 sc := ps.makeSimpleSchema(dt, parquet.Repetitions.Required) 1137 data := ps.writeColumn(mem, sc, values) 1138 ps.readAndCheckSingleColumnFile(mem, data, values) 1139 }) 1140 } 1141 } 1142 1143 func (ps *ParquetIOTestSuite) roundTripTable(mem memory.Allocator, expected arrow.Table, storeSchema bool) { 1144 var buf bytes.Buffer 1145 var props pqarrow.ArrowWriterProperties 1146 if storeSchema { 1147 props = pqarrow.NewArrowWriterProperties(pqarrow.WithStoreSchema(), pqarrow.WithAllocator(mem)) 1148 } else { 1149 props = pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem)) 1150 } 1151 1152 writeProps := parquet.NewWriterProperties(parquet.WithAllocator(mem)) 1153 ps.Require().NoError(pqarrow.WriteTable(expected, &buf, expected.NumRows(), writeProps, props)) 1154 1155 reader := ps.createReader(mem, buf.Bytes()) 1156 defer reader.ParquetReader().Close() 1157 1158 tbl := ps.readTable(reader) 1159 defer tbl.Release() 1160 1161 ps.Equal(expected.NumCols(), tbl.NumCols()) 1162 ps.Equal(expected.NumRows(), tbl.NumRows()) 1163 1164 exChunk := expected.Column(0).Data() 1165 tblChunk := tbl.Column(0).Data() 1166 1167 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 1168 exc := exChunk.Chunk(0) 1169 tbc := tblChunk.Chunk(0) 1170 ps.Truef(array.ApproxEqual(exc, tbc), "expected: %T %s\ngot: %T %s", exc, exc, tbc, tbc) 1171 } 1172 1173 func makeEmptyListsArray(size int) arrow.Array { 1174 // allocate an offsets buffer with only zeros 1175 offsetsNbytes := arrow.Int32Traits.BytesRequired(size + 1) 1176 offsetsBuffer := make([]byte, offsetsNbytes) 1177 1178 childBuffers := []*memory.Buffer{nil, nil} 1179 childData := array.NewData(arrow.PrimitiveTypes.Float32, 0, childBuffers, nil, 0, 0) 1180 defer childData.Release() 1181 buffers := []*memory.Buffer{nil, memory.NewBufferBytes(offsetsBuffer)} 1182 arrayData := array.NewData(arrow.ListOf(childData.DataType()), size, buffers, []arrow.ArrayData{childData}, 0, 0) 1183 defer arrayData.Release() 1184 return array.MakeFromData(arrayData) 1185 } 1186 1187 func makeListArray(values arrow.Array, size, nullcount int) arrow.Array { 1188 nonNullEntries := size - nullcount - 1 1189 lengthPerEntry := values.Len() / nonNullEntries 1190 1191 offsets := make([]byte, arrow.Int32Traits.BytesRequired(size+1)) 1192 offsetsArr := arrow.Int32Traits.CastFromBytes(offsets) 1193 1194 nullBitmap := make([]byte, int(bitutil.BytesForBits(int64(size)))) 1195 1196 curOffset := 0 1197 for i := 0; i < size; i++ { 1198 offsetsArr[i] = int32(curOffset) 1199 if !(((i % 2) == 0) && ((i / 2) < nullcount)) { 1200 // non-null list (list with index 1 is always empty) 1201 bitutil.SetBit(nullBitmap, i) 1202 if i != 1 { 1203 curOffset += lengthPerEntry 1204 } 1205 } 1206 } 1207 offsetsArr[size] = int32(values.Len()) 1208 1209 listData := array.NewData(arrow.ListOf(values.DataType()), size, 1210 []*memory.Buffer{memory.NewBufferBytes(nullBitmap), memory.NewBufferBytes(offsets)}, 1211 []arrow.ArrayData{values.Data()}, nullcount, 0) 1212 defer listData.Release() 1213 return array.NewListData(listData) 1214 } 1215 1216 func prepareEmptyListsTable(size int) arrow.Table { 1217 lists := makeEmptyListsArray(size) 1218 defer lists.Release() 1219 chunked := arrow.NewChunked(lists.DataType(), []arrow.Array{lists}) 1220 defer chunked.Release() 1221 return makeSimpleTable(chunked, true) 1222 } 1223 1224 func prepareListTable(dt arrow.DataType, size int, nullableLists bool, nullableElems bool, nullCount int) arrow.Table { 1225 nc := nullCount 1226 if !nullableElems { 1227 nc = 0 1228 } 1229 values := testutils.RandomNullable(dt, size*size, nc) 1230 defer values.Release() 1231 // also test that slice offsets are respected 1232 values = array.NewSlice(values, 5, int64(values.Len())) 1233 defer values.Release() 1234 1235 if !nullableLists { 1236 nullCount = 0 1237 } 1238 lists := makeListArray(values, size, nullCount) 1239 defer lists.Release() 1240 1241 chunked := arrow.NewChunked(lists.DataType(), []arrow.Array{lists}) 1242 defer chunked.Release() 1243 1244 return makeSimpleTable(array.NewChunkedSlice(chunked, 3, int64(size)), nullableLists) 1245 } 1246 1247 func prepareListOfListTable(dt arrow.DataType, size, nullCount int, nullableParentLists, nullableLists, nullableElems bool) arrow.Table { 1248 nc := nullCount 1249 if !nullableElems { 1250 nc = 0 1251 } 1252 1253 values := testutils.RandomNullable(dt, size*6, nc) 1254 defer values.Release() 1255 1256 if nullableLists { 1257 nc = nullCount 1258 } else { 1259 nc = 0 1260 } 1261 1262 lists := makeListArray(values, size*3, nc) 1263 defer lists.Release() 1264 1265 if !nullableParentLists { 1266 nullCount = 0 1267 } 1268 1269 parentLists := makeListArray(lists, size, nullCount) 1270 defer parentLists.Release() 1271 1272 chunked := arrow.NewChunked(parentLists.DataType(), []arrow.Array{parentLists}) 1273 defer chunked.Release() 1274 1275 return makeSimpleTable(chunked, nullableParentLists) 1276 } 1277 1278 func (ps *ParquetIOTestSuite) TestSingleEmptyListsColumnReadWrite() { 1279 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1280 defer mem.AssertSize(ps.T(), 0) 1281 1282 expected := prepareEmptyListsTable(smallSize) 1283 defer expected.Release() 1284 buf := writeTableToBuffer(ps.T(), mem, expected, smallSize, pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 1285 defer buf.Release() 1286 1287 reader := ps.createReader(mem, buf.Bytes()) 1288 tbl := ps.readTable(reader) 1289 defer tbl.Release() 1290 1291 ps.EqualValues(expected.NumCols(), tbl.NumCols()) 1292 ps.EqualValues(expected.NumRows(), tbl.NumRows()) 1293 1294 exChunk := expected.Column(0).Data() 1295 tblChunk := tbl.Column(0).Data() 1296 1297 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 1298 ps.True(array.Equal(exChunk.Chunk(0), tblChunk.Chunk(0))) 1299 } 1300 1301 func (ps *ParquetIOTestSuite) TestSingleColumnOptionalReadWrite() { 1302 for _, dt := range fullTypeList { 1303 ps.Run(dt.Name(), func() { 1304 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1305 defer mem.AssertSize(ps.T(), 0) 1306 1307 values := testutils.RandomNullable(dt, smallSize, 10) 1308 defer values.Release() 1309 sc := ps.makeSimpleSchema(dt, parquet.Repetitions.Optional) 1310 data := ps.writeColumn(mem, sc, values) 1311 ps.readAndCheckSingleColumnFile(mem, data, values) 1312 }) 1313 } 1314 } 1315 1316 func (ps *ParquetIOTestSuite) TestSingleNullableListNullableColumnReadWrite() { 1317 for _, dt := range fullTypeList { 1318 ps.Run(dt.Name(), func() { 1319 expected := prepareListTable(dt, smallSize, true, true, 10) 1320 defer expected.Release() 1321 ps.roundTripTable(memory.DefaultAllocator, expected, false) 1322 }) 1323 } 1324 } 1325 1326 func (ps *ParquetIOTestSuite) TestSingleRequiredListNullableColumnReadWrite() { 1327 for _, dt := range fullTypeList { 1328 ps.Run(dt.Name(), func() { 1329 expected := prepareListTable(dt, smallSize, false, true, 10) 1330 defer expected.Release() 1331 ps.roundTripTable(memory.DefaultAllocator, expected, false) 1332 }) 1333 } 1334 } 1335 1336 func (ps *ParquetIOTestSuite) TestSingleNullableListRequiredColumnReadWrite() { 1337 for _, dt := range fullTypeList { 1338 ps.Run(dt.Name(), func() { 1339 expected := prepareListTable(dt, smallSize, true, false, 10) 1340 defer expected.Release() 1341 ps.roundTripTable(memory.DefaultAllocator, expected, false) 1342 }) 1343 } 1344 } 1345 1346 func (ps *ParquetIOTestSuite) TestSingleRequiredListRequiredColumnReadWrite() { 1347 for _, dt := range fullTypeList { 1348 ps.Run(dt.Name(), func() { 1349 expected := prepareListTable(dt, smallSize, false, false, 0) 1350 defer expected.Release() 1351 ps.roundTripTable(memory.DefaultAllocator, expected, false) 1352 }) 1353 } 1354 } 1355 1356 func (ps *ParquetIOTestSuite) TestSingleNullableListRequiredListRequiredColumnReadWrite() { 1357 for _, dt := range fullTypeList { 1358 ps.Run(dt.Name(), func() { 1359 expected := prepareListOfListTable(dt, smallSize, 2, true, false, false) 1360 defer expected.Release() 1361 ps.roundTripTable(memory.DefaultAllocator, expected, false) 1362 }) 1363 } 1364 } 1365 1366 func (ps *ParquetIOTestSuite) TestSimpleStruct() { 1367 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1368 defer mem.AssertSize(ps.T(), 0) 1369 1370 links := arrow.StructOf(arrow.Field{Name: "Backward", Type: arrow.PrimitiveTypes.Int64, Nullable: true}, 1371 arrow.Field{Name: "Forward", Type: arrow.PrimitiveTypes.Int64, Nullable: true}) 1372 1373 bldr := array.NewStructBuilder(mem, links) 1374 defer bldr.Release() 1375 1376 backBldr := bldr.FieldBuilder(0).(*array.Int64Builder) 1377 forwardBldr := bldr.FieldBuilder(1).(*array.Int64Builder) 1378 1379 bldr.Append(true) 1380 backBldr.AppendNull() 1381 forwardBldr.Append(20) 1382 1383 bldr.Append(true) 1384 backBldr.Append(10) 1385 forwardBldr.Append(40) 1386 1387 data := bldr.NewArray() 1388 defer data.Release() 1389 1390 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{{Name: "links", Type: links}}, nil), 1391 []arrow.Column{*arrow.NewColumn(arrow.Field{Name: "links", Type: links}, arrow.NewChunked(links, []arrow.Array{data}))}, -1) 1392 defer data.Release() // NewChunked 1393 defer tbl.Release() 1394 1395 ps.roundTripTable(mem, tbl, false) 1396 } 1397 1398 func (ps *ParquetIOTestSuite) TestSingleColumnNullableStruct() { 1399 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1400 defer mem.AssertSize(ps.T(), 0) 1401 1402 links := arrow.StructOf(arrow.Field{Name: "Backward", Type: arrow.PrimitiveTypes.Int64, Nullable: true}) 1403 bldr := array.NewStructBuilder(mem, links) 1404 defer bldr.Release() 1405 1406 backBldr := bldr.FieldBuilder(0).(*array.Int64Builder) 1407 1408 bldr.AppendNull() 1409 bldr.Append(true) 1410 backBldr.Append(10) 1411 1412 data := bldr.NewArray() 1413 defer data.Release() 1414 1415 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{{Name: "links", Type: links, Nullable: true}}, nil), 1416 []arrow.Column{*arrow.NewColumn(arrow.Field{Name: "links", Type: links, Nullable: true}, arrow.NewChunked(links, []arrow.Array{data}))}, -1) 1417 defer data.Release() // NewChunked 1418 defer tbl.Release() 1419 1420 ps.roundTripTable(mem, tbl, false) 1421 } 1422 1423 func (ps *ParquetIOTestSuite) TestNestedRequiredFieldStruct() { 1424 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1425 defer mem.AssertSize(ps.T(), 0) 1426 1427 intField := arrow.Field{Name: "int_array", Type: arrow.PrimitiveTypes.Int32} 1428 intBldr := array.NewInt32Builder(mem) 1429 defer intBldr.Release() 1430 intBldr.AppendValues([]int32{0, 1, 2, 3, 4, 5, 7, 8}, nil) 1431 1432 intArr := intBldr.NewArray() 1433 defer intArr.Release() 1434 1435 validity := memory.NewBufferBytes([]byte{0xCC}) 1436 defer validity.Release() 1437 1438 structField := arrow.Field{Name: "root", Type: arrow.StructOf(intField), Nullable: true} 1439 structData := array.NewData(structField.Type, 8, []*memory.Buffer{validity}, []arrow.ArrayData{intArr.Data()}, 4, 0) 1440 defer structData.Release() 1441 stData := array.NewStructData(structData) 1442 defer stData.Release() 1443 1444 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{structField}, nil), 1445 []arrow.Column{*arrow.NewColumn(structField, 1446 arrow.NewChunked(structField.Type, []arrow.Array{stData}))}, -1) 1447 defer stData.Release() // NewChunked 1448 defer tbl.Release() 1449 1450 ps.roundTripTable(mem, tbl, false) 1451 } 1452 1453 func (ps *ParquetIOTestSuite) TestNestedNullableField() { 1454 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1455 defer mem.AssertSize(ps.T(), 0) 1456 1457 intField := arrow.Field{Name: "int_array", Type: arrow.PrimitiveTypes.Int32, Nullable: true} 1458 intBldr := array.NewInt32Builder(mem) 1459 defer intBldr.Release() 1460 intBldr.AppendValues([]int32{0, 1, 2, 3, 4, 5, 7, 8}, []bool{true, false, true, false, true, true, false, true}) 1461 1462 intArr := intBldr.NewArray() 1463 defer intArr.Release() 1464 1465 validity := memory.NewBufferBytes([]byte{0xCC}) 1466 defer validity.Release() 1467 1468 structField := arrow.Field{Name: "root", Type: arrow.StructOf(intField), Nullable: true} 1469 data := array.NewData(structField.Type, 8, []*memory.Buffer{validity}, []arrow.ArrayData{intArr.Data()}, 4, 0) 1470 defer data.Release() 1471 stData := array.NewStructData(data) 1472 defer stData.Release() 1473 1474 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{structField}, nil), 1475 []arrow.Column{*arrow.NewColumn(structField, 1476 arrow.NewChunked(structField.Type, []arrow.Array{stData}))}, -1) 1477 defer stData.Release() // NewChunked 1478 defer tbl.Release() 1479 1480 ps.roundTripTable(mem, tbl, false) 1481 } 1482 1483 func (ps *ParquetIOTestSuite) TestNestedEmptyList() { 1484 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1485 defer mem.AssertSize(ps.T(), 0) 1486 1487 bldr := array.NewStructBuilder(mem, arrow.StructOf( 1488 arrow.Field{ 1489 Name: "root", 1490 Type: arrow.StructOf( 1491 arrow.Field{ 1492 Name: "child1", 1493 Type: arrow.ListOf(arrow.StructOf( 1494 arrow.Field{ 1495 Name: "child2", 1496 Type: arrow.ListOf(arrow.StructOf( 1497 arrow.Field{ 1498 Name: "name", 1499 Type: arrow.BinaryTypes.String, 1500 }, 1501 )), 1502 }, 1503 )), 1504 }, 1505 ), 1506 }, 1507 )) 1508 defer bldr.Release() 1509 1510 rootBldr := bldr.FieldBuilder(0).(*array.StructBuilder) 1511 child1Bldr := rootBldr.FieldBuilder(0).(*array.ListBuilder) 1512 child1ElBldr := child1Bldr.ValueBuilder().(*array.StructBuilder) 1513 child2Bldr := child1ElBldr.FieldBuilder(0).(*array.ListBuilder) 1514 leafBldr := child2Bldr.ValueBuilder().(*array.StructBuilder) 1515 nameBldr := leafBldr.FieldBuilder(0).(*array.StringBuilder) 1516 1517 // target structure 8 times 1518 // { 1519 // "root": { 1520 // "child1": [ 1521 // { "child2": [{ "name": "foo" }] }, 1522 // { "child2": [] } 1523 // ] 1524 // } 1525 // } 1526 1527 for i := 0; i < 8; i++ { 1528 bldr.Append(true) 1529 rootBldr.Append(true) 1530 child1Bldr.Append(true) 1531 1532 child1ElBldr.Append(true) 1533 child2Bldr.Append(true) 1534 leafBldr.Append(true) 1535 nameBldr.Append("foo") 1536 1537 child1ElBldr.Append(true) 1538 child2Bldr.Append(true) 1539 } 1540 1541 arr := bldr.NewArray() 1542 defer arr.Release() 1543 1544 field := arrow.Field{Name: "x", Type: arr.DataType(), Nullable: true} 1545 expected := array.NewTableFromSlice(arrow.NewSchema([]arrow.Field{field}, nil), [][]arrow.Array{{arr}}) 1546 defer expected.Release() 1547 1548 ps.roundTripTable(mem, expected, false) 1549 } 1550 1551 func (ps *ParquetIOTestSuite) TestCanonicalNestedRoundTrip() { 1552 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1553 defer mem.AssertSize(ps.T(), 0) 1554 1555 docIdField := arrow.Field{Name: "DocID", Type: arrow.PrimitiveTypes.Int64} 1556 linksField := arrow.Field{Name: "Links", Type: arrow.StructOf( 1557 arrow.Field{Name: "Backward", Type: arrow.ListOf(arrow.PrimitiveTypes.Int64)}, 1558 arrow.Field{Name: "Forward", Type: arrow.ListOf(arrow.PrimitiveTypes.Int64)}, 1559 ), Nullable: true} 1560 1561 nameStruct := arrow.StructOf( 1562 arrow.Field{Name: "Language", Nullable: true, Type: arrow.ListOf( 1563 arrow.StructOf(arrow.Field{Name: "Code", Type: arrow.BinaryTypes.String}, 1564 arrow.Field{Name: "Country", Type: arrow.BinaryTypes.String, Nullable: true}))}, 1565 arrow.Field{Name: "Url", Type: arrow.BinaryTypes.String, Nullable: true}) 1566 1567 nameField := arrow.Field{Name: "Name", Type: arrow.ListOf(nameStruct)} 1568 sc := arrow.NewSchema([]arrow.Field{docIdField, linksField, nameField}, nil) 1569 1570 docIDArr, _, err := array.FromJSON(mem, docIdField.Type, strings.NewReader("[10, 20]")) 1571 ps.Require().NoError(err) 1572 defer docIDArr.Release() 1573 1574 linksIDArr, _, err := array.FromJSON(mem, linksField.Type, strings.NewReader(`[{"Backward":[], "Forward":[20, 40, 60]}, {"Backward":[10, 30], "Forward": [80]}]`)) 1575 ps.Require().NoError(err) 1576 defer linksIDArr.Release() 1577 1578 nameArr, _, err := array.FromJSON(mem, nameField.Type, strings.NewReader(` 1579 [[{"Language": [{"Code": "en_us", "Country": "us"}, 1580 {"Code": "en_us", "Country": null}], 1581 "Url": "http://A"}, 1582 {"Url": "http://B", "Language": null}, 1583 {"Language": [{"Code": "en-gb", "Country": "gb"}], "Url": null}], 1584 [{"Url": "http://C", "Language": null}]]`)) 1585 ps.Require().NoError(err) 1586 defer nameArr.Release() 1587 1588 expected := array.NewTable(sc, []arrow.Column{ 1589 *arrow.NewColumn(docIdField, arrow.NewChunked(docIdField.Type, []arrow.Array{docIDArr})), 1590 *arrow.NewColumn(linksField, arrow.NewChunked(linksField.Type, []arrow.Array{linksIDArr})), 1591 *arrow.NewColumn(nameField, arrow.NewChunked(nameField.Type, []arrow.Array{nameArr})), 1592 }, 2) 1593 defer docIDArr.Release() // NewChunked 1594 defer linksIDArr.Release() // NewChunked 1595 defer nameArr.Release() // NewChunked 1596 defer expected.Release() 1597 1598 ps.roundTripTable(mem, expected, false) 1599 } 1600 1601 func (ps *ParquetIOTestSuite) TestFixedSizeList() { 1602 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1603 defer mem.AssertSize(ps.T(), 0) 1604 1605 bldr := array.NewFixedSizeListBuilder(mem, 3, arrow.PrimitiveTypes.Int16) 1606 defer bldr.Release() 1607 1608 vb := bldr.ValueBuilder().(*array.Int16Builder) 1609 1610 bldr.AppendValues([]bool{true, true, true}) 1611 vb.AppendValues([]int16{1, 2, 3, 4, 5, 6, 7, 8, 9}, nil) 1612 1613 data := bldr.NewArray() 1614 defer data.Release() // NewArray 1615 1616 field := arrow.Field{Name: "root", Type: data.DataType(), Nullable: true} 1617 cnk := arrow.NewChunked(field.Type, []arrow.Array{data}) 1618 defer data.Release() // NewChunked 1619 1620 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{field}, nil), []arrow.Column{*arrow.NewColumn(field, cnk)}, -1) 1621 defer cnk.Release() // NewColumn 1622 defer tbl.Release() 1623 1624 ps.roundTripTable(mem, tbl, true) 1625 } 1626 1627 func (ps *ParquetIOTestSuite) TestNull() { 1628 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1629 defer mem.AssertSize(ps.T(), 0) 1630 1631 bldr := array.NewNullBuilder(mem) 1632 defer bldr.Release() 1633 1634 bldr.AppendNull() 1635 bldr.AppendNull() 1636 bldr.AppendNull() 1637 1638 data := bldr.NewArray() 1639 defer data.Release() 1640 1641 field := arrow.Field{Name: "x", Type: data.DataType(), Nullable: true} 1642 expected := array.NewTable( 1643 arrow.NewSchema([]arrow.Field{field}, nil), 1644 []arrow.Column{*arrow.NewColumn(field, arrow.NewChunked(field.Type, []arrow.Array{data}))}, 1645 -1, 1646 ) 1647 1648 ps.roundTripTable(mem, expected, true) 1649 } 1650 1651 // ARROW-17169 1652 func (ps *ParquetIOTestSuite) TestNullableListOfStruct() { 1653 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1654 defer mem.AssertSize(ps.T(), 0) 1655 1656 bldr := array.NewListBuilder(mem, arrow.StructOf( 1657 arrow.Field{Name: "a", Type: arrow.PrimitiveTypes.Int32}, 1658 arrow.Field{Name: "b", Type: arrow.BinaryTypes.String}, 1659 )) 1660 defer bldr.Release() 1661 1662 stBldr := bldr.ValueBuilder().(*array.StructBuilder) 1663 aBldr := stBldr.FieldBuilder(0).(*array.Int32Builder) 1664 bBldr := stBldr.FieldBuilder(1).(*array.StringBuilder) 1665 1666 for i := 0; i < 320; i++ { 1667 if i%5 == 0 { 1668 bldr.AppendNull() 1669 continue 1670 } 1671 bldr.Append(true) 1672 for j := 0; j < 4; j++ { 1673 stBldr.Append(true) 1674 aBldr.Append(int32(i + j)) 1675 bBldr.Append(strconv.Itoa(i + j)) 1676 } 1677 } 1678 1679 arr := bldr.NewArray() 1680 defer arr.Release() 1681 1682 field := arrow.Field{Name: "x", Type: arr.DataType(), Nullable: true} 1683 expected := array.NewTable(arrow.NewSchema([]arrow.Field{field}, nil), 1684 []arrow.Column{*arrow.NewColumn(field, arrow.NewChunked(field.Type, []arrow.Array{arr}))}, -1) 1685 defer arr.Release() // NewChunked 1686 defer expected.Release() 1687 1688 ps.roundTripTable(mem, expected, false) 1689 } 1690 1691 func (ps *ParquetIOTestSuite) TestStructWithListOfNestedStructs() { 1692 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1693 defer mem.AssertSize(ps.T(), 0) 1694 1695 bldr := array.NewStructBuilder(mem, arrow.StructOf( 1696 arrow.Field{ 1697 Nullable: true, 1698 Name: "l", 1699 Type: arrow.ListOf(arrow.StructOf( 1700 arrow.Field{ 1701 Nullable: true, 1702 Name: "a", 1703 Type: arrow.StructOf( 1704 arrow.Field{ 1705 Nullable: true, 1706 Name: "b", 1707 Type: arrow.BinaryTypes.String, 1708 }, 1709 ), 1710 }, 1711 )), 1712 }, 1713 )) 1714 defer bldr.Release() 1715 1716 lBldr := bldr.FieldBuilder(0).(*array.ListBuilder) 1717 stBldr := lBldr.ValueBuilder().(*array.StructBuilder) 1718 aBldr := stBldr.FieldBuilder(0).(*array.StructBuilder) 1719 bBldr := aBldr.FieldBuilder(0).(*array.StringBuilder) 1720 1721 bldr.AppendNull() 1722 bldr.Append(true) 1723 lBldr.Append(true) 1724 for i := 0; i < 8; i++ { 1725 stBldr.Append(true) 1726 aBldr.Append(true) 1727 bBldr.Append(strconv.Itoa(i)) 1728 } 1729 1730 arr := bldr.NewArray() 1731 defer arr.Release() 1732 1733 field := arrow.Field{Name: "x", Type: arr.DataType(), Nullable: true} 1734 expected := array.NewTable(arrow.NewSchema([]arrow.Field{field}, nil), 1735 []arrow.Column{*arrow.NewColumn(field, arrow.NewChunked(field.Type, []arrow.Array{arr}))}, -1) 1736 defer arr.Release() // NewChunked 1737 defer expected.Release() 1738 1739 ps.roundTripTable(mem, expected, false) 1740 } 1741 1742 func TestParquetArrowIO(t *testing.T) { 1743 suite.Run(t, new(ParquetIOTestSuite)) 1744 } 1745 1746 func TestBufferedRecWrite(t *testing.T) { 1747 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1748 defer mem.AssertSize(t, 0) 1749 1750 sc := arrow.NewSchema([]arrow.Field{ 1751 {Name: "f32", Type: arrow.PrimitiveTypes.Float32, Nullable: true}, 1752 {Name: "i32", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, 1753 {Name: "struct_i64_f64", Type: arrow.StructOf( 1754 arrow.Field{Name: "i64", Type: arrow.PrimitiveTypes.Int64, Nullable: true}, 1755 arrow.Field{Name: "f64", Type: arrow.PrimitiveTypes.Float64, Nullable: true})}, 1756 }, nil) 1757 1758 structData := array.NewData(sc.Field(2).Type, SIZELEN, 1759 []*memory.Buffer{nil, nil}, 1760 []arrow.ArrayData{testutils.RandomNullable(arrow.PrimitiveTypes.Int64, SIZELEN, 0).Data(), testutils.RandomNullable(arrow.PrimitiveTypes.Float64, SIZELEN, 0).Data()}, 0, 0) 1761 defer structData.Release() 1762 cols := []arrow.Array{ 1763 testutils.RandomNullable(sc.Field(0).Type, SIZELEN, SIZELEN/5), 1764 testutils.RandomNullable(sc.Field(1).Type, SIZELEN, SIZELEN/5), 1765 array.NewStructData(structData), 1766 } 1767 1768 rec := array.NewRecord(sc, cols, SIZELEN) 1769 defer rec.Release() 1770 1771 var ( 1772 buf bytes.Buffer 1773 ) 1774 1775 wr, err := pqarrow.NewFileWriter(sc, &buf, 1776 parquet.NewWriterProperties(parquet.WithCompression(compress.Codecs.Snappy), parquet.WithDictionaryDefault(false), parquet.WithDataPageSize(100*1024)), 1777 pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 1778 require.NoError(t, err) 1779 1780 p1 := rec.NewSlice(0, SIZELEN/2) 1781 defer p1.Release() 1782 require.NoError(t, wr.WriteBuffered(p1)) 1783 1784 p2 := rec.NewSlice(SIZELEN/2, SIZELEN) 1785 defer p2.Release() 1786 require.NoError(t, wr.WriteBuffered(p2)) 1787 1788 wr.Close() 1789 1790 rdr, err := file.NewParquetReader(bytes.NewReader(buf.Bytes())) 1791 assert.NoError(t, err) 1792 1793 assert.EqualValues(t, 1, rdr.NumRowGroups()) 1794 assert.EqualValues(t, SIZELEN, rdr.NumRows()) 1795 rdr.Close() 1796 1797 tbl, err := pqarrow.ReadTable(context.Background(), bytes.NewReader(buf.Bytes()), nil, pqarrow.ArrowReadProperties{}, nil) 1798 assert.NoError(t, err) 1799 defer tbl.Release() 1800 1801 assert.EqualValues(t, SIZELEN, tbl.NumRows()) 1802 } 1803 1804 func (ps *ParquetIOTestSuite) TestArrowMapTypeRoundTrip() { 1805 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1806 defer mem.AssertSize(ps.T(), 0) 1807 1808 bldr := array.NewMapBuilder(mem, arrow.BinaryTypes.String, arrow.PrimitiveTypes.Int32, false) 1809 defer bldr.Release() 1810 1811 kb := bldr.KeyBuilder().(*array.StringBuilder) 1812 ib := bldr.ItemBuilder().(*array.Int32Builder) 1813 1814 bldr.Append(true) 1815 kb.AppendValues([]string{"Fee", "Fi", "Fo", "Fum"}, nil) 1816 ib.AppendValues([]int32{1, 2, 3, 4}, nil) 1817 1818 bldr.Append(true) 1819 kb.AppendValues([]string{"Fee", "Fi", "Fo"}, nil) 1820 ib.AppendValues([]int32{5, 4, 3}, nil) 1821 1822 bldr.AppendNull() 1823 1824 bldr.Append(true) 1825 kb.AppendValues([]string{"Fo", "Fi", "Fee"}, nil) 1826 ib.AppendValues([]int32{-1, 2, 3}, []bool{false, true, true}) 1827 1828 arr := bldr.NewArray() 1829 defer arr.Release() 1830 1831 fld := arrow.Field{Name: "mapped", Type: arr.DataType(), Nullable: true} 1832 cnk := arrow.NewChunked(arr.DataType(), []arrow.Array{arr}) 1833 defer arr.Release() // NewChunked 1834 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{fld}, nil), []arrow.Column{*arrow.NewColumn(fld, cnk)}, -1) 1835 defer cnk.Release() // NewColumn 1836 defer tbl.Release() 1837 1838 ps.roundTripTable(mem, tbl, true) 1839 } 1840 1841 func (ps *ParquetIOTestSuite) TestArrowExtensionTypeRoundTrip() { 1842 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1843 defer mem.AssertSize(ps.T(), 0) 1844 1845 extBuilder := array.NewExtensionBuilder(mem, types.NewUUIDType()) 1846 defer extBuilder.Release() 1847 builder := types.NewUUIDBuilder(extBuilder) 1848 builder.Append(uuid.New()) 1849 arr := builder.NewArray() 1850 defer arr.Release() 1851 1852 fld := arrow.Field{Name: "uuid", Type: arr.DataType(), Nullable: true} 1853 cnk := arrow.NewChunked(arr.DataType(), []arrow.Array{arr}) 1854 defer arr.Release() // NewChunked 1855 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{fld}, nil), []arrow.Column{*arrow.NewColumn(fld, cnk)}, -1) 1856 defer cnk.Release() // NewColumn 1857 defer tbl.Release() 1858 1859 ps.roundTripTable(mem, tbl, true) 1860 } 1861 1862 func (ps *ParquetIOTestSuite) TestArrowUnknownExtensionTypeRoundTrip() { 1863 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1864 defer mem.AssertSize(ps.T(), 0) 1865 1866 var written, expected arrow.Table 1867 1868 { 1869 // Prepare `written` table with the extension type registered. 1870 extType := types.NewUUIDType() 1871 bldr := array.NewExtensionBuilder(mem, extType) 1872 defer bldr.Release() 1873 1874 bldr.Builder.(*array.FixedSizeBinaryBuilder).AppendValues( 1875 [][]byte{nil, []byte("abcdefghijklmno0"), []byte("abcdefghijklmno1"), []byte("abcdefghijklmno2")}, 1876 []bool{false, true, true, true}) 1877 1878 arr := bldr.NewArray() 1879 defer arr.Release() 1880 1881 if arrow.GetExtensionType("uuid") != nil { 1882 ps.NoError(arrow.UnregisterExtensionType("uuid")) 1883 } 1884 1885 fld := arrow.Field{Name: "uuid", Type: arr.DataType(), Nullable: true} 1886 cnk := arrow.NewChunked(arr.DataType(), []arrow.Array{arr}) 1887 defer arr.Release() // NewChunked 1888 written = array.NewTable(arrow.NewSchema([]arrow.Field{fld}, nil), []arrow.Column{*arrow.NewColumn(fld, cnk)}, -1) 1889 defer cnk.Release() // NewColumn 1890 defer written.Release() 1891 } 1892 1893 { 1894 // Prepare `expected` table with the extension type unregistered in the underlying type. 1895 bldr := array.NewFixedSizeBinaryBuilder(mem, &arrow.FixedSizeBinaryType{ByteWidth: 16}) 1896 defer bldr.Release() 1897 bldr.AppendValues( 1898 [][]byte{nil, []byte("abcdefghijklmno0"), []byte("abcdefghijklmno1"), []byte("abcdefghijklmno2")}, 1899 []bool{false, true, true, true}) 1900 1901 arr := bldr.NewArray() 1902 defer arr.Release() 1903 1904 fld := arrow.Field{Name: "uuid", Type: arr.DataType(), Nullable: true} 1905 cnk := arrow.NewChunked(arr.DataType(), []arrow.Array{arr}) 1906 defer arr.Release() // NewChunked 1907 expected = array.NewTable(arrow.NewSchema([]arrow.Field{fld}, nil), []arrow.Column{*arrow.NewColumn(fld, cnk)}, -1) 1908 defer cnk.Release() // NewColumn 1909 defer expected.Release() 1910 } 1911 1912 // sanity check before going deeper 1913 ps.Equal(expected.NumCols(), written.NumCols()) 1914 ps.Equal(expected.NumRows(), written.NumRows()) 1915 1916 // just like roundTripTable() but different written vs. expected tables 1917 var buf bytes.Buffer 1918 props := pqarrow.NewArrowWriterProperties(pqarrow.WithStoreSchema(), pqarrow.WithAllocator(mem)) 1919 1920 writeProps := parquet.NewWriterProperties(parquet.WithAllocator(mem)) 1921 ps.Require().NoError(pqarrow.WriteTable(written, &buf, written.NumRows(), writeProps, props)) 1922 1923 reader := ps.createReader(mem, buf.Bytes()) 1924 defer reader.ParquetReader().Close() 1925 1926 tbl := ps.readTable(reader) 1927 defer tbl.Release() 1928 1929 ps.Equal(expected.NumCols(), tbl.NumCols()) 1930 ps.Equal(expected.NumRows(), tbl.NumRows()) 1931 1932 exChunk := expected.Column(0).Data() 1933 tblChunk := tbl.Column(0).Data() 1934 1935 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 1936 exc := exChunk.Chunk(0) 1937 tbc := tblChunk.Chunk(0) 1938 ps.Truef(array.Equal(exc, tbc), "expected: %T %s\ngot: %T %s", exc, exc, tbc, tbc) 1939 1940 expectedMd := arrow.MetadataFrom(map[string]string{ 1941 ipc.ExtensionTypeKeyName: "uuid", 1942 ipc.ExtensionMetadataKeyName: "uuid-serialized", 1943 "PARQUET:field_id": "-1", 1944 }) 1945 ps.Truef(expectedMd.Equal(tbl.Column(0).Field().Metadata), "expected: %v\ngot: %v", expectedMd, tbl.Column(0).Field().Metadata) 1946 } 1947 1948 func TestWriteTableMemoryAllocation(t *testing.T) { 1949 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 1950 sc := arrow.NewSchema([]arrow.Field{ 1951 {Name: "f32", Type: arrow.PrimitiveTypes.Float32, Nullable: true}, 1952 {Name: "i32", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, 1953 {Name: "struct_i64_f64", Type: arrow.StructOf( 1954 arrow.Field{Name: "i64", Type: arrow.PrimitiveTypes.Int64, Nullable: true}, 1955 arrow.Field{Name: "f64", Type: arrow.PrimitiveTypes.Float64, Nullable: true})}, 1956 {Name: "arr_i64", Type: arrow.ListOf(arrow.PrimitiveTypes.Int64)}, 1957 {Name: "uuid", Type: types.NewUUIDType(), Nullable: true}, 1958 }, nil) 1959 1960 bld := array.NewRecordBuilder(mem, sc) 1961 bld.Field(0).(*array.Float32Builder).Append(1.0) 1962 bld.Field(1).(*array.Int32Builder).Append(1) 1963 sbld := bld.Field(2).(*array.StructBuilder) 1964 sbld.Append(true) 1965 sbld.FieldBuilder(0).(*array.Int64Builder).Append(1) 1966 sbld.FieldBuilder(1).(*array.Float64Builder).Append(1.0) 1967 abld := bld.Field(3).(*array.ListBuilder) 1968 abld.Append(true) 1969 abld.ValueBuilder().(*array.Int64Builder).Append(2) 1970 bld.Field(4).(*types.UUIDBuilder).Append(uuid.MustParse("00000000-0000-0000-0000-000000000001")) 1971 1972 rec := bld.NewRecord() 1973 bld.Release() 1974 1975 var buf bytes.Buffer 1976 wr, err := pqarrow.NewFileWriter(sc, &buf, 1977 parquet.NewWriterProperties(parquet.WithCompression(compress.Codecs.Snappy)), 1978 pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 1979 require.NoError(t, err) 1980 1981 require.NoError(t, wr.Write(rec)) 1982 rec.Release() 1983 wr.Close() 1984 1985 require.Zero(t, mem.CurrentAlloc()) 1986 }