github.com/segmentio/parquet-go@v0.0.0-20230712180008-5d42db8f0d47/row_group_test.go (about) 1 package parquet_test 2 3 import ( 4 "bytes" 5 "io" 6 "reflect" 7 "sort" 8 "testing" 9 10 "github.com/segmentio/parquet-go" 11 ) 12 13 func sortedRowGroup(options []parquet.RowGroupOption, rows ...interface{}) parquet.RowGroup { 14 buf := parquet.NewBuffer(options...) 15 for _, row := range rows { 16 buf.Write(row) 17 } 18 sort.Stable(buf) 19 return buf 20 } 21 22 type Person struct { 23 FirstName utf8string 24 LastName utf8string 25 Age int 26 } 27 28 type LastNameOnly struct { 29 LastName utf8string 30 } 31 32 func newPeopleBuffer(people []Person) parquet.RowGroup { 33 buffer := parquet.NewBuffer() 34 for i := range people { 35 buffer.Write(&people[i]) 36 } 37 return buffer 38 } 39 40 func newPeopleFile(people []Person) parquet.RowGroup { 41 buffer := new(bytes.Buffer) 42 writer := parquet.NewWriter(buffer) 43 for i := range people { 44 writer.Write(&people[i]) 45 } 46 writer.Close() 47 reader := bytes.NewReader(buffer.Bytes()) 48 f, err := parquet.OpenFile(reader, reader.Size()) 49 if err != nil { 50 panic(err) 51 } 52 return f.RowGroups()[0] 53 } 54 55 func TestSeekToRow(t *testing.T) { 56 for _, config := range []struct { 57 name string 58 newRowGroup func([]Person) parquet.RowGroup 59 }{ 60 {name: "buffer", newRowGroup: newPeopleBuffer}, 61 {name: "file", newRowGroup: newPeopleFile}, 62 } { 63 t.Run(config.name, func(t *testing.T) { testSeekToRow(t, config.newRowGroup) }) 64 } 65 } 66 67 func testSeekToRow(t *testing.T, newRowGroup func([]Person) parquet.RowGroup) { 68 err := quickCheck(func(people []Person) bool { 69 if len(people) == 0 { // TODO: fix creation of empty parquet files 70 return true 71 } 72 rowGroup := newRowGroup(people) 73 rows := rowGroup.Rows() 74 rbuf := make([]parquet.Row, 1) 75 pers := Person{} 76 schema := parquet.SchemaOf(&pers) 77 defer rows.Close() 78 79 for i := range people { 80 if err := rows.SeekToRow(int64(i)); err != nil { 81 t.Errorf("seeking to row %d: %+v", i, err) 82 return false 83 } 84 if _, err := rows.ReadRows(rbuf); err != nil { 85 t.Errorf("reading row %d: %+v", i, err) 86 return false 87 } 88 if err := schema.Reconstruct(&pers, rbuf[0]); err != nil { 89 t.Errorf("deconstructing row %d: %+v", i, err) 90 return false 91 } 92 if !reflect.DeepEqual(&pers, &people[i]) { 93 t.Errorf("row %d mismatch", i) 94 return false 95 } 96 } 97 98 return true 99 }) 100 if err != nil { 101 t.Error(err) 102 } 103 } 104 105 func selfRowGroup(rowGroup parquet.RowGroup) parquet.RowGroup { 106 return rowGroup 107 } 108 109 func fileRowGroup(rowGroup parquet.RowGroup) parquet.RowGroup { 110 buffer := new(bytes.Buffer) 111 writer := parquet.NewWriter(buffer) 112 if _, err := writer.WriteRowGroup(rowGroup); err != nil { 113 panic(err) 114 } 115 if err := writer.Close(); err != nil { 116 panic(err) 117 } 118 reader := bytes.NewReader(buffer.Bytes()) 119 f, err := parquet.OpenFile(reader, reader.Size()) 120 if err != nil { 121 panic(err) 122 } 123 return f.RowGroups()[0] 124 } 125 126 func TestWriteRowGroupClosesRows(t *testing.T) { 127 var rows []*wrappedRows 128 rg := wrappedRowGroup{ 129 RowGroup: newPeopleFile([]Person{{}}), 130 rowsCallback: func(r parquet.Rows) parquet.Rows { 131 wrapped := &wrappedRows{Rows: r} 132 rows = append(rows, wrapped) 133 return wrapped 134 }, 135 } 136 writer := parquet.NewWriter(io.Discard) 137 if _, err := writer.WriteRowGroup(rg); err != nil { 138 t.Fatal(err) 139 } 140 if err := writer.Close(); err != nil { 141 t.Fatal(err) 142 } 143 for _, r := range rows { 144 if !r.closed { 145 t.Fatal("rows not closed") 146 } 147 } 148 }