storj.io/minio@v0.0.0-20230509071714-0cbc90f649b1/pkg/s3select/internal/parquet-go/writer.go (about) 1 /* 2 * Minio Cloud Storage, (C) 2019 Minio, Inc. 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package parquet 18 19 import ( 20 "context" 21 "encoding/binary" 22 "fmt" 23 "io" 24 25 "git.apache.org/thrift.git/lib/go/thrift" 26 27 "storj.io/minio/pkg/s3select/internal/parquet-go/data" 28 "storj.io/minio/pkg/s3select/internal/parquet-go/gen-go/parquet" 29 "storj.io/minio/pkg/s3select/internal/parquet-go/schema" 30 ) 31 32 const ( 33 defaultPageSize = 8 * 1024 // 8 KiB 34 defaultRowGroupSize = 128 * 1024 * 1024 // 128 MiB 35 ) 36 37 // Writer - represents parquet writer. 38 type Writer struct { 39 PageSize int64 40 RowGroupSize int64 41 CompressionType parquet.CompressionCodec 42 43 writeCloser io.WriteCloser 44 numRows int64 45 offset int64 46 footer *parquet.FileMetaData 47 schemaTree *schema.Tree 48 valueElements []*schema.Element 49 columnDataMap map[string]*data.Column 50 rowGroupCount int 51 } 52 53 func (writer *Writer) writeData() (err error) { 54 if writer.numRows == 0 { 55 return nil 56 } 57 58 var chunks []*data.ColumnChunk 59 for _, element := range writer.valueElements { 60 name := element.PathInTree 61 columnData, found := writer.columnDataMap[name] 62 if !found { 63 continue 64 } 65 66 columnChunk := columnData.Encode(element) 67 chunks = append(chunks, columnChunk) 68 } 69 70 rowGroup := data.NewRowGroup(chunks, writer.numRows, writer.offset) 71 72 for _, chunk := range chunks { 73 if _, err = writer.writeCloser.Write(chunk.Data()); err != nil { 74 return err 75 } 76 77 writer.offset += chunk.DataLen() 78 } 79 80 writer.footer.RowGroups = append(writer.footer.RowGroups, rowGroup) 81 writer.footer.NumRows += writer.numRows 82 83 writer.numRows = 0 84 writer.columnDataMap = nil 85 return nil 86 } 87 88 // WriteJSON - writes a record represented in JSON. 89 func (writer *Writer) WriteJSON(recordData []byte) (err error) { 90 columnDataMap, err := data.UnmarshalJSON(recordData, writer.schemaTree) 91 if err != nil { 92 return err 93 } 94 95 return writer.Write(columnDataMap) 96 } 97 98 // Write - writes a record represented in map. 99 func (writer *Writer) Write(record map[string]*data.Column) (err error) { 100 if writer.columnDataMap == nil { 101 writer.columnDataMap = record 102 } else { 103 for name, columnData := range record { 104 var found bool 105 var element *schema.Element 106 for _, element = range writer.valueElements { 107 if element.PathInTree == name { 108 found = true 109 break 110 } 111 } 112 113 if !found { 114 return fmt.Errorf("%v is not value column", name) 115 } 116 117 writer.columnDataMap[name].Merge(columnData) 118 } 119 } 120 121 writer.numRows++ 122 if writer.numRows == int64(writer.rowGroupCount) { 123 return writer.writeData() 124 } 125 126 return nil 127 } 128 129 func (writer *Writer) finalize() (err error) { 130 if err = writer.writeData(); err != nil { 131 return err 132 } 133 134 ts := thrift.NewTSerializer() 135 ts.Protocol = thrift.NewTCompactProtocolFactory().GetProtocol(ts.Transport) 136 footerBuf, err := ts.Write(context.TODO(), writer.footer) 137 if err != nil { 138 return err 139 } 140 141 if _, err = writer.writeCloser.Write(footerBuf); err != nil { 142 return err 143 } 144 145 footerSizeBuf := make([]byte, 4) 146 binary.LittleEndian.PutUint32(footerSizeBuf, uint32(len(footerBuf))) 147 148 if _, err = writer.writeCloser.Write(footerSizeBuf); err != nil { 149 return err 150 } 151 152 _, err = writer.writeCloser.Write([]byte("PAR1")) 153 return err 154 } 155 156 // Close - finalizes and closes writer. If any pending records are available, they are written here. 157 func (writer *Writer) Close() (err error) { 158 if err = writer.finalize(); err != nil { 159 return err 160 } 161 162 return writer.writeCloser.Close() 163 } 164 165 // NewWriter - creates new parquet writer. Binary data of rowGroupCount records are written to writeCloser. 166 func NewWriter(writeCloser io.WriteCloser, schemaTree *schema.Tree, rowGroupCount int) (*Writer, error) { 167 if _, err := writeCloser.Write([]byte("PAR1")); err != nil { 168 return nil, err 169 } 170 171 schemaList, valueElements, err := schemaTree.ToParquetSchema() 172 if err != nil { 173 return nil, err 174 } 175 176 footer := parquet.NewFileMetaData() 177 footer.Version = 1 178 footer.Schema = schemaList 179 180 return &Writer{ 181 PageSize: defaultPageSize, 182 RowGroupSize: defaultRowGroupSize, 183 CompressionType: parquet.CompressionCodec_SNAPPY, 184 185 writeCloser: writeCloser, 186 offset: 4, 187 footer: footer, 188 schemaTree: schemaTree, 189 valueElements: valueElements, 190 rowGroupCount: rowGroupCount, 191 }, nil 192 }