-
Notifications
You must be signed in to change notification settings - Fork 0
/
reader.go
372 lines (323 loc) · 8.76 KB
/
reader.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
package parquet
import (
"context"
"encoding/binary"
"io"
"reflect"
"strings"
"sync"
"github.com/apache/thrift/lib/go/thrift"
"github.com/stdiopt/parquet/internal/parquet"
)
// Reader contains methods and information to read parquet files.
type Reader struct {
SchemaHandler *SchemaHandler
NP int64 // parallel number
Footer *parquet.FileMetaData
PFile File
ColumnBuffers map[string]*columnBufferType
// One reader can only read one type objects
ObjType reflect.Type
ObjPartialType reflect.Type
}
// NewReader Create a parquet reader: obj is a object with schema tags or a JSON schema string.
func NewReader(pFile File, obj interface{}, np int64) (*Reader, error) {
var err error
res := new(Reader)
res.NP = np
res.PFile = pFile
if err = res.ReadFooter(); err != nil {
return nil, err
}
res.ColumnBuffers = make(map[string]*columnBufferType)
if obj != nil {
if sa, ok := obj.(string); ok {
err = res.SetSchemaHandlerFromJSON(sa)
return res, err
} else if sa, ok := obj.([]*parquet.SchemaElement); ok {
res.SchemaHandler = NewSchemaHandlerFromSchemaList(sa)
} else {
if res.SchemaHandler, err = NewSchemaHandlerFromStruct(obj); err != nil {
return res, err
}
res.ObjType = reflect.TypeOf(obj).Elem()
}
} else {
res.SchemaHandler = NewSchemaHandlerFromSchemaList(res.Footer.Schema)
}
res.RenameSchema()
for i := 0; i < len(res.SchemaHandler.SchemaElements); i++ {
schema := res.SchemaHandler.SchemaElements[i]
if schema.GetNumChildren() != 0 {
continue
}
pathStr := res.SchemaHandler.IndexMap[int32(i)]
if res.ColumnBuffers[pathStr], err = newColumnBuffer(pFile, res.Footer, res.SchemaHandler, pathStr); err != nil {
return res, err
}
}
return res, nil
}
// SetSchemaHandlerFromJSON reads a json schema.
func (pr *Reader) SetSchemaHandlerFromJSON(jsonSchema string) error {
var err error
if pr.SchemaHandler, err = NewSchemaHandlerFromJSON(jsonSchema); err != nil {
return err
}
pr.RenameSchema()
for i := 0; i < len(pr.SchemaHandler.SchemaElements); i++ {
schemaElement := pr.SchemaHandler.SchemaElements[i]
if schemaElement.GetNumChildren() != 0 {
continue
}
pathStr := pr.SchemaHandler.IndexMap[int32(i)]
if pr.ColumnBuffers[pathStr], err = newColumnBuffer(pr.PFile, pr.Footer, pr.SchemaHandler, pathStr); err != nil {
return err
}
}
return nil
}
// RenameSchema renames schema name to inname.
func (pr *Reader) RenameSchema() {
for i := 0; i < len(pr.SchemaHandler.Infos); i++ {
pr.Footer.Schema[i].Name = pr.SchemaHandler.Infos[i].InName
}
for _, rowGroup := range pr.Footer.RowGroups {
for _, chunk := range rowGroup.Columns {
exPath := make([]string, 0)
exPath = append(exPath, pr.SchemaHandler.GetRootExName())
exPath = append(exPath, chunk.MetaData.GetPathInSchema()...)
exPathStr := pathToStr(exPath)
inPathStr := pr.SchemaHandler.ExPathToInPath[exPathStr]
inPath := strToPath(inPathStr)[1:]
chunk.MetaData.PathInSchema = inPath
}
}
}
// GetNumRows return number of rows.
func (pr *Reader) GetNumRows() int64 {
return pr.Footer.GetNumRows()
}
// GetFooterSize returns the footer size.
func (pr *Reader) GetFooterSize() (uint32, error) {
var err error
buf := make([]byte, 4)
if _, err = pr.PFile.Seek(-8, io.SeekEnd); err != nil {
return 0, err
}
if _, err = io.ReadFull(pr.PFile, buf); err != nil {
return 0, err
}
size := binary.LittleEndian.Uint32(buf)
return size, err
}
// ReadFooter reads the footer from parquet file.
func (pr *Reader) ReadFooter() error {
size, err := pr.GetFooterSize()
if err != nil {
return err
}
if _, err = pr.PFile.Seek(-(int64)(8+size), io.SeekEnd); err != nil {
return err
}
pr.Footer = parquet.NewFileMetaData()
sr := thrift.NewStreamTransportR(pr.PFile)
protocol := thrift.NewTCompactProtocolConf(sr, nil)
// pf := thrift.NewTCompactProtocolFactory()
// protocol := pf.GetProtocol(thrift.NewStreamTransportR(pr.PFile))
return pr.Footer.Read(context.TODO(), protocol)
}
// SkipRows of parquet file.
func (pr *Reader) SkipRows(num int64) error {
var err error
if num <= 0 {
return nil
}
doneChan := make(chan int, pr.NP)
taskChan := make(chan string, len(pr.SchemaHandler.ValueColumns))
stopChan := make(chan int)
for _, pathStr := range pr.SchemaHandler.ValueColumns {
if _, ok := pr.ColumnBuffers[pathStr]; ok {
continue
}
pr.ColumnBuffers[pathStr], err = newColumnBuffer(
pr.PFile,
pr.Footer,
pr.SchemaHandler,
pathStr,
)
if err != nil {
return err
}
}
for i := int64(0); i < pr.NP; i++ {
go func() {
for {
select {
case <-stopChan:
return
case pathStr := <-taskChan:
cb := pr.ColumnBuffers[pathStr]
cb.SkipRows(num)
doneChan <- 0
}
}
}()
}
for key := range pr.ColumnBuffers {
taskChan <- key
}
for i := 0; i < len(pr.ColumnBuffers); i++ {
<-doneChan
}
for i := int64(0); i < pr.NP; i++ {
stopChan <- 0
}
return err
}
// Read rows of parquet file and unmarshal all to dst.
func (pr *Reader) Read(dstInterface interface{}) error {
return pr.read(dstInterface, "")
}
// ReadByNumber read maxReadNumber objects.
func (pr *Reader) ReadByNumber(maxReadNumber int) ([]interface{}, error) {
var err error
if pr.ObjType == nil {
if pr.ObjType, err = pr.SchemaHandler.GetType(pr.SchemaHandler.GetRootInName()); err != nil {
return nil, err
}
}
vs := reflect.MakeSlice(reflect.SliceOf(pr.ObjType), maxReadNumber, maxReadNumber)
res := reflect.New(vs.Type())
res.Elem().Set(vs)
if err = pr.Read(res.Interface()); err != nil {
return nil, err
}
ln := res.Elem().Len()
ret := make([]interface{}, ln)
for i := 0; i < ln; i++ {
ret[i] = res.Elem().Index(i).Interface()
}
return ret, nil
}
// ReadPartial read rows of parquet file and unmarshal all to dst.
func (pr *Reader) ReadPartial(dstInterface interface{}, prefixPath string) error {
prefixPath, err := pr.SchemaHandler.ConvertToInPathStr(prefixPath)
if err != nil {
return err
}
return pr.read(dstInterface, prefixPath)
}
// ReadPartialByNumber read maxReadNumber partial objects.
func (pr *Reader) ReadPartialByNumber(maxReadNumber int, prefixPath string) ([]interface{}, error) {
var err error
if pr.ObjPartialType == nil {
if pr.ObjPartialType, err = pr.SchemaHandler.GetType(prefixPath); err != nil {
return nil, err
}
}
vs := reflect.MakeSlice(reflect.SliceOf(pr.ObjPartialType), maxReadNumber, maxReadNumber)
res := reflect.New(vs.Type())
res.Elem().Set(vs)
if err = pr.ReadPartial(res.Interface(), prefixPath); err != nil {
return nil, err
}
ln := res.Elem().Len()
ret := make([]interface{}, ln)
for i := 0; i < ln; i++ {
ret[i] = res.Elem().Index(i).Interface()
}
return ret, nil
}
// read rows of parquet file with a prefixPath.
func (pr *Reader) read(dstInterface interface{}, prefixPath string) error {
var err error
tmap := make(map[string]*layoutTable)
locker := new(sync.Mutex)
ot := reflect.TypeOf(dstInterface).Elem().Elem()
num := reflect.ValueOf(dstInterface).Elem().Len()
if num <= 0 {
return nil
}
doneChan := make(chan int, pr.NP)
taskChan := make(chan string, len(pr.ColumnBuffers))
stopChan := make(chan int)
for i := int64(0); i < pr.NP; i++ {
go func() {
for {
select {
case <-stopChan:
return
case pathStr := <-taskChan:
cb := pr.ColumnBuffers[pathStr]
table, _ := cb.ReadRows(int64(num))
locker.Lock()
if _, ok := tmap[pathStr]; ok {
tmap[pathStr].Merge(table)
} else {
tmap[pathStr] = newLayoutTableFromTable(table)
tmap[pathStr].Merge(table)
}
locker.Unlock()
doneChan <- 0
}
}
}()
}
readNum := 0
for key := range pr.ColumnBuffers {
if strings.HasPrefix(key, prefixPath) {
taskChan <- key
readNum++
}
}
for i := 0; i < readNum; i++ {
<-doneChan
}
for i := int64(0); i < pr.NP; i++ {
stopChan <- 0
}
dstList := make([]interface{}, pr.NP)
delta := (int64(num) + pr.NP - 1) / pr.NP
var wg sync.WaitGroup
for c := int64(0); c < pr.NP; c++ {
bgn := c * delta
end := bgn + delta
if end > int64(num) {
end = int64(num)
}
if bgn >= int64(num) {
bgn, end = int64(num), int64(num)
}
wg.Add(1)
go func(b, e, index int) {
defer func() {
wg.Done()
}()
dstList[index] = reflect.New(reflect.SliceOf(ot)).Interface()
if err2 := unmarshal(&tmap, b, e, dstList[index], pr.SchemaHandler, prefixPath); err2 != nil {
err = err2
}
}(int(bgn), int(end), int(c))
}
wg.Wait()
dstValue := reflect.ValueOf(dstInterface).Elem()
dstValue.SetLen(0)
for _, dst := range dstList {
dstValue.Set(reflect.AppendSlice(dstValue, reflect.ValueOf(dst).Elem()))
}
return err
}
// Close closes the parquer file.
func (pr *Reader) Close() error {
var merr error
for _, cb := range pr.ColumnBuffers {
if cb == nil {
continue
}
if err := cb.PFile.Close(); err != nil {
merr = err
}
}
return merr
}