/
writer.go
191 lines (157 loc) · 4.71 KB
/
writer.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
/*
* Minio Cloud Storage, (C) 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package parquet
import (
"context"
"encoding/binary"
"fmt"
"io"
"git.apache.org/thrift.git/lib/go/thrift"
"github.com/cdbarbosa/clone/pkg/s3select/internal/parquet-go/data"
"github.com/cdbarbosa/clone/pkg/s3select/internal/parquet-go/gen-go/parquet"
"github.com/cdbarbosa/clone/pkg/s3select/internal/parquet-go/schema"
)
const (
defaultPageSize = 8 * 1024 // 8 KiB
defaultRowGroupSize = 128 * 1024 * 1024 // 128 MiB
)
// Writer - represents parquet writer.
type Writer struct {
PageSize int64
RowGroupSize int64
CompressionType parquet.CompressionCodec
writeCloser io.WriteCloser
numRows int64
offset int64
footer *parquet.FileMetaData
schemaTree *schema.Tree
valueElements []*schema.Element
columnDataMap map[string]*data.Column
rowGroupCount int
}
func (writer *Writer) writeData() (err error) {
if writer.numRows == 0 {
return nil
}
var chunks []*data.ColumnChunk
for _, element := range writer.valueElements {
name := element.PathInTree
columnData, found := writer.columnDataMap[name]
if !found {
continue
}
columnChunk := columnData.Encode(element)
chunks = append(chunks, columnChunk)
}
rowGroup := data.NewRowGroup(chunks, writer.numRows, writer.offset)
for _, chunk := range chunks {
if _, err = writer.writeCloser.Write(chunk.Data()); err != nil {
return err
}
writer.offset += chunk.DataLen()
}
writer.footer.RowGroups = append(writer.footer.RowGroups, rowGroup)
writer.footer.NumRows += writer.numRows
writer.numRows = 0
writer.columnDataMap = nil
return nil
}
// WriteJSON - writes a record represented in JSON.
func (writer *Writer) WriteJSON(recordData []byte) (err error) {
columnDataMap, err := data.UnmarshalJSON(recordData, writer.schemaTree)
if err != nil {
return err
}
return writer.Write(columnDataMap)
}
// Write - writes a record represented in map.
func (writer *Writer) Write(record map[string]*data.Column) (err error) {
if writer.columnDataMap == nil {
writer.columnDataMap = record
} else {
for name, columnData := range record {
var found bool
var element *schema.Element
for _, element = range writer.valueElements {
if element.PathInTree == name {
found = true
break
}
}
if !found {
return fmt.Errorf("%v is not value column", name)
}
writer.columnDataMap[name].Merge(columnData)
}
}
writer.numRows++
if writer.numRows == int64(writer.rowGroupCount) {
return writer.writeData()
}
return nil
}
func (writer *Writer) finalize() (err error) {
if err = writer.writeData(); err != nil {
return err
}
ts := thrift.NewTSerializer()
ts.Protocol = thrift.NewTCompactProtocolFactory().GetProtocol(ts.Transport)
footerBuf, err := ts.Write(context.TODO(), writer.footer)
if err != nil {
return err
}
if _, err = writer.writeCloser.Write(footerBuf); err != nil {
return err
}
footerSizeBuf := make([]byte, 4)
binary.LittleEndian.PutUint32(footerSizeBuf, uint32(len(footerBuf)))
if _, err = writer.writeCloser.Write(footerSizeBuf); err != nil {
return err
}
_, err = writer.writeCloser.Write([]byte("PAR1"))
return err
}
// Close - finalizes and closes writer. If any pending records are available, they are written here.
func (writer *Writer) Close() (err error) {
if err = writer.finalize(); err != nil {
return err
}
return writer.writeCloser.Close()
}
// NewWriter - creates new parquet writer. Binary data of rowGroupCount records are written to writeCloser.
func NewWriter(writeCloser io.WriteCloser, schemaTree *schema.Tree, rowGroupCount int) (*Writer, error) {
if _, err := writeCloser.Write([]byte("PAR1")); err != nil {
return nil, err
}
schemaList, valueElements, err := schemaTree.ToParquetSchema()
if err != nil {
return nil, err
}
footer := parquet.NewFileMetaData()
footer.Version = 1
footer.Schema = schemaList
return &Writer{
PageSize: defaultPageSize,
RowGroupSize: defaultRowGroupSize,
CompressionType: parquet.CompressionCodec_SNAPPY,
writeCloser: writeCloser,
offset: 4,
footer: footer,
schemaTree: schemaTree,
valueElements: valueElements,
rowGroupCount: rowGroupCount,
}, nil
}