github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/mydump/parquet_parser.go (about) 1 package mydump 2 3 import ( 4 "bytes" 5 "context" 6 "fmt" 7 "io" 8 "reflect" 9 "strings" 10 "time" 11 12 "github.com/pingcap/br/pkg/storage" 13 "github.com/pingcap/errors" 14 "github.com/pingcap/tidb-lightning/lightning/log" 15 "github.com/pingcap/tidb/types" 16 "github.com/xitongsys/parquet-go/parquet" 17 preader "github.com/xitongsys/parquet-go/reader" 18 "github.com/xitongsys/parquet-go/source" 19 "go.uber.org/zap" 20 ) 21 22 const ( 23 batchReadRowSize = 32 24 25 // if a parquet if small than this threshold, parquet will load the whole file in a byte slice to 26 // optimize the read performance 27 smallParquetFileThreshold = 256 * 1024 * 1024 28 ) 29 30 type ParquetParser struct { 31 Reader *preader.ParquetReader 32 columns []string 33 columnMetas []*parquet.SchemaElement 34 rows []interface{} 35 readRows int64 36 curStart int64 37 curIndex int 38 lastRow Row 39 logger log.Logger 40 } 41 42 // readerWrapper is a used for implement `source.ParquetFile` 43 type readerWrapper struct { 44 ReadSeekCloser 45 store storage.ExternalStorage 46 ctx context.Context 47 // current file path 48 path string 49 } 50 51 func (r *readerWrapper) Write(p []byte) (n int, err error) { 52 return 0, errors.New("unsupported operation") 53 } 54 55 func (r *readerWrapper) Open(name string) (source.ParquetFile, error) { 56 if len(name) == 0 { 57 name = r.path 58 } 59 reader, err := r.store.Open(r.ctx, name) 60 if err != nil { 61 return nil, errors.Trace(err) 62 } 63 return &readerWrapper{ 64 ReadSeekCloser: reader, 65 store: r.store, 66 ctx: r.ctx, 67 path: name, 68 }, nil 69 } 70 func (r *readerWrapper) Create(name string) (source.ParquetFile, error) { 71 return nil, errors.New("unsupported operation") 72 } 73 74 // bytesReaderWrapper is a wrapper of bytes.Reader used for implement `source.ParquetFile` 75 type bytesReaderWrapper struct { 76 *bytes.Reader 77 rawBytes []byte 78 // current file path 79 path string 80 } 81 82 func (r *bytesReaderWrapper) Close() error { 83 return nil 84 } 85 86 func (r *bytesReaderWrapper) Create(name string) (source.ParquetFile, error) { 87 return nil, errors.New("unsupported operation") 88 } 89 90 func (r *bytesReaderWrapper) Write(p []byte) (n int, err error) { 91 return 0, errors.New("unsupported operation") 92 } 93 94 func (r *bytesReaderWrapper) Open(name string) (source.ParquetFile, error) { 95 if len(name) > 0 && name != r.path { 96 panic(fmt.Sprintf("Open with a different name is not supported! current: '%s', new: '%s'", r.path, name)) 97 } 98 return &bytesReaderWrapper{ 99 Reader: bytes.NewReader(r.rawBytes), 100 rawBytes: r.rawBytes, 101 path: r.path, 102 }, nil 103 } 104 105 func OpenParquetReader( 106 ctx context.Context, 107 store storage.ExternalStorage, 108 path string, 109 size int64, 110 ) (source.ParquetFile, error) { 111 if size <= smallParquetFileThreshold { 112 fileBytes, err := store.Read(ctx, path) 113 if err != nil { 114 return nil, err 115 } 116 return &bytesReaderWrapper{ 117 Reader: bytes.NewReader(fileBytes), 118 rawBytes: fileBytes, 119 path: path, 120 }, nil 121 } 122 123 r, err := store.Open(ctx, path) 124 if err != nil { 125 return nil, err 126 } 127 return &readerWrapper{ 128 ReadSeekCloser: r, 129 store: store, 130 ctx: ctx, 131 path: path, 132 }, nil 133 } 134 135 // a special func to fetch parquet file row count fast. 136 func ReadParquetFileRowCount( 137 ctx context.Context, 138 store storage.ExternalStorage, 139 r storage.ReadSeekCloser, 140 path string, 141 ) (int64, error) { 142 wrapper := &readerWrapper{ 143 ReadSeekCloser: r, 144 store: store, 145 ctx: ctx, 146 path: path, 147 } 148 var err error 149 res := new(preader.ParquetReader) 150 res.NP = 1 151 res.PFile = wrapper 152 if err = res.ReadFooter(); err != nil { 153 return 0, err 154 } 155 numRows := res.Footer.NumRows 156 if err = wrapper.Close(); err != nil { 157 return 0, err 158 } 159 return numRows, nil 160 } 161 162 func NewParquetParser( 163 ctx context.Context, 164 store storage.ExternalStorage, 165 r storage.ReadSeekCloser, 166 path string, 167 ) (*ParquetParser, error) { 168 // check to avoid wrapping twice 169 wrapper, ok := r.(source.ParquetFile) 170 if !ok { 171 wrapper = &readerWrapper{ 172 ReadSeekCloser: r, 173 store: store, 174 ctx: ctx, 175 path: path, 176 } 177 } 178 179 // FIXME: need to bench what the best value for the concurrent reader number 180 reader, err := preader.NewParquetReader(wrapper, nil, 2) 181 if err != nil { 182 return nil, errors.Trace(err) 183 } 184 185 columns := make([]string, 0, len(reader.Footer.Schema)-1) 186 columnMetas := make([]*parquet.SchemaElement, 0, len(reader.Footer.Schema)-1) 187 for _, c := range reader.SchemaHandler.SchemaElements { 188 if c.GetNumChildren() == 0 { 189 // NOTE: the SchemaElement.Name is capitalized, SchemaHandler.Infos.ExName is the raw column name 190 // though in this context, there is no difference between these two fields 191 columns = append(columns, strings.ToLower(c.Name)) 192 columnMetas = append(columnMetas, c) 193 } 194 } 195 196 return &ParquetParser{ 197 Reader: reader, 198 columns: columns, 199 columnMetas: columnMetas, 200 logger: log.L(), 201 }, nil 202 } 203 204 // Pos returns the currently row number of the parquet file 205 func (pp *ParquetParser) Pos() (pos int64, rowID int64) { 206 return pp.curStart + int64(pp.curIndex), pp.lastRow.RowID 207 } 208 209 func (pp *ParquetParser) SetPos(pos int64, rowID int64) error { 210 if pos < pp.curStart { 211 panic("don't support seek back yet") 212 } 213 pp.lastRow.RowID = rowID 214 215 if pos < pp.curStart+int64(len(pp.rows)) { 216 pp.curIndex = int(pos - pp.curStart) 217 pp.readRows = pos 218 return nil 219 } 220 221 if pos > pp.curStart+int64(len(pp.rows)) { 222 if err := pp.Reader.SkipRows(pos - pp.curStart - int64(len(pp.rows))); err != nil { 223 return errors.Trace(err) 224 } 225 } 226 pp.curStart = pos 227 pp.readRows = pos 228 pp.curIndex = 0 229 if len(pp.rows) > 0 { 230 pp.rows = pp.rows[:0] 231 } 232 233 return nil 234 } 235 236 func (pp *ParquetParser) Close() error { 237 pp.Reader.ReadStop() 238 return pp.Reader.PFile.Close() 239 } 240 241 func (pp *ParquetParser) ReadRow() error { 242 pp.lastRow.RowID++ 243 if pp.curIndex >= len(pp.rows) { 244 if pp.readRows >= pp.Reader.GetNumRows() { 245 return io.EOF 246 } 247 count := batchReadRowSize 248 if pp.Reader.GetNumRows()-pp.readRows < int64(count) { 249 count = int(pp.Reader.GetNumRows() - pp.readRows) 250 } 251 252 var err error 253 pp.rows, err = pp.Reader.ReadByNumber(count) 254 if err != nil { 255 return errors.Trace(err) 256 } 257 pp.curStart = pp.readRows 258 pp.readRows += int64(len(pp.rows)) 259 pp.curIndex = 0 260 } 261 262 row := pp.rows[pp.curIndex] 263 pp.curIndex++ 264 265 v := reflect.ValueOf(row) 266 length := v.NumField() 267 if cap(pp.lastRow.Row) < length { 268 pp.lastRow.Row = make([]types.Datum, length) 269 } else { 270 pp.lastRow.Row = pp.lastRow.Row[:length] 271 } 272 for i := 0; i < length; i++ { 273 setDatumValue(&pp.lastRow.Row[i], v.Field(i), pp.columnMetas[i]) 274 } 275 return nil 276 } 277 278 // convert a parquet value to Datum 279 // 280 // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md 281 func setDatumValue(d *types.Datum, v reflect.Value, meta *parquet.SchemaElement) { 282 switch v.Kind() { 283 case reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64: 284 d.SetUint64(v.Uint()) 285 case reflect.Int8, reflect.Int16: 286 d.SetInt64(v.Int()) 287 case reflect.Int32, reflect.Int64: 288 setDatumByInt(d, v.Int(), meta) 289 case reflect.String: 290 d.SetString(v.String(), "") 291 case reflect.Float32, reflect.Float64: 292 d.SetFloat64(v.Float()) 293 case reflect.Ptr: 294 if v.IsNil() { 295 d.SetNull() 296 } else { 297 setDatumValue(d, v.Elem(), meta) 298 } 299 default: 300 log.L().Fatal("unknown value", zap.Stringer("kind", v.Kind()), 301 zap.String("type", v.Type().Name()), zap.Reflect("value", v.Interface())) 302 } 303 } 304 305 // when the value type is int32/int64, convert to value to target logical type in tidb 306 func setDatumByInt(d *types.Datum, v int64, meta *parquet.SchemaElement) { 307 if meta.ConvertedType == nil { 308 d.SetInt64(v) 309 return 310 } 311 switch *meta.ConvertedType { 312 // decimal 313 case parquet.ConvertedType_DECIMAL: 314 minLen := *meta.Scale + 1 315 if v < 0 { 316 minLen++ 317 } 318 val := fmt.Sprintf("%0*d", minLen, v) 319 dotIndex := len(val) - int(*meta.Scale) 320 d.SetString(val[:dotIndex]+"."+val[dotIndex:], "") 321 case parquet.ConvertedType_DATE: 322 dateStr := time.Unix(v*86400, 0).Format("2006-01-02") 323 d.SetString(dateStr, "") 324 // convert all timestamp types (datetime/timestamp) to string 325 case parquet.ConvertedType_TIMESTAMP_MICROS: 326 dateStr := time.Unix(v/1e6, (v%1e6)*1e3).Format("2006-01-02 15:04:05.999") 327 d.SetString(dateStr, "") 328 case parquet.ConvertedType_TIMESTAMP_MILLIS: 329 dateStr := time.Unix(v/1e3, (v%1e3)*1e6).Format("2006-01-02 15:04:05.999") 330 d.SetString(dateStr, "") 331 // covert time types to string 332 case parquet.ConvertedType_TIME_MILLIS, parquet.ConvertedType_TIME_MICROS: 333 if *meta.ConvertedType == parquet.ConvertedType_TIME_MICROS { 334 v /= 1e3 335 } 336 millis := v % 1e3 337 v /= 1e3 338 sec := v % 60 339 v /= 60 340 min := v % 60 341 v /= 60 342 d.SetString(fmt.Sprintf("%d:%d:%d.%3d", v, min, sec, millis), "") 343 default: 344 d.SetInt64(v) 345 } 346 } 347 348 func (pp *ParquetParser) LastRow() Row { 349 return pp.lastRow 350 } 351 352 func (pp *ParquetParser) RecycleRow(row Row) { 353 } 354 355 // Columns returns the _lower-case_ column names corresponding to values in 356 // the LastRow. 357 func (pp *ParquetParser) Columns() []string { 358 return pp.columns 359 } 360 361 // SetColumns set restored column names to parser 362 func (pp *ParquetParser) SetColumns(cols []string) { 363 // just do nothing 364 } 365 366 func (pp *ParquetParser) SetLogger(l log.Logger) { 367 pp.logger = l 368 }