github.com/pingcap/br@v5.3.0-alpha.0.20220125034240-ec59c7b6ce30+incompatible/pkg/lightning/mydump/parquet_parser.go (about) 1 package mydump 2 3 import ( 4 "bytes" 5 "context" 6 "fmt" 7 "io" 8 "math/big" 9 "reflect" 10 "strings" 11 "time" 12 13 "github.com/pingcap/errors" 14 "github.com/pingcap/tidb/types" 15 "github.com/xitongsys/parquet-go/parquet" 16 preader "github.com/xitongsys/parquet-go/reader" 17 "github.com/xitongsys/parquet-go/source" 18 "go.uber.org/zap" 19 20 "github.com/pingcap/br/pkg/lightning/log" 21 "github.com/pingcap/br/pkg/storage" 22 ) 23 24 const ( 25 batchReadRowSize = 32 26 27 // if a parquet if small than this threshold, parquet will load the whole file in a byte slice to 28 // optimize the read performance 29 smallParquetFileThreshold = 256 * 1024 * 1024 30 ) 31 32 type ParquetParser struct { 33 Reader *preader.ParquetReader 34 columns []string 35 columnMetas []*parquet.SchemaElement 36 rows []interface{} 37 readRows int64 38 curStart int64 39 curIndex int 40 lastRow Row 41 logger log.Logger 42 } 43 44 // readerWrapper is a used for implement `source.ParquetFile` 45 type readerWrapper struct { 46 ReadSeekCloser 47 store storage.ExternalStorage 48 ctx context.Context 49 // current file path 50 path string 51 } 52 53 func (r *readerWrapper) Write(p []byte) (n int, err error) { 54 return 0, errors.New("unsupported operation") 55 } 56 57 func (r *readerWrapper) Open(name string) (source.ParquetFile, error) { 58 if len(name) == 0 { 59 name = r.path 60 } 61 reader, err := r.store.Open(r.ctx, name) 62 if err != nil { 63 return nil, errors.Trace(err) 64 } 65 return &readerWrapper{ 66 ReadSeekCloser: reader, 67 store: r.store, 68 ctx: r.ctx, 69 path: name, 70 }, nil 71 } 72 73 func (r *readerWrapper) Create(name string) (source.ParquetFile, error) { 74 return nil, errors.New("unsupported operation") 75 } 76 77 // bytesReaderWrapper is a wrapper of bytes.Reader used for implement `source.ParquetFile` 78 type bytesReaderWrapper struct { 79 *bytes.Reader 80 rawBytes []byte 81 // current file path 82 path string 83 } 84 85 func (r *bytesReaderWrapper) Close() error { 86 return nil 87 } 88 89 func (r *bytesReaderWrapper) Create(name string) (source.ParquetFile, error) { 90 return nil, errors.New("unsupported operation") 91 } 92 93 func (r *bytesReaderWrapper) Write(p []byte) (n int, err error) { 94 return 0, errors.New("unsupported operation") 95 } 96 97 func (r *bytesReaderWrapper) Open(name string) (source.ParquetFile, error) { 98 if len(name) > 0 && name != r.path { 99 panic(fmt.Sprintf("Open with a different name is not supported! current: '%s', new: '%s'", r.path, name)) 100 } 101 return &bytesReaderWrapper{ 102 Reader: bytes.NewReader(r.rawBytes), 103 rawBytes: r.rawBytes, 104 path: r.path, 105 }, nil 106 } 107 108 func OpenParquetReader( 109 ctx context.Context, 110 store storage.ExternalStorage, 111 path string, 112 size int64, 113 ) (source.ParquetFile, error) { 114 if size <= smallParquetFileThreshold { 115 fileBytes, err := store.ReadFile(ctx, path) 116 if err != nil { 117 return nil, err 118 } 119 return &bytesReaderWrapper{ 120 Reader: bytes.NewReader(fileBytes), 121 rawBytes: fileBytes, 122 path: path, 123 }, nil 124 } 125 126 r, err := store.Open(ctx, path) 127 if err != nil { 128 return nil, err 129 } 130 return &readerWrapper{ 131 ReadSeekCloser: r, 132 store: store, 133 ctx: ctx, 134 path: path, 135 }, nil 136 } 137 138 // a special func to fetch parquet file row count fast. 139 func ReadParquetFileRowCount( 140 ctx context.Context, 141 store storage.ExternalStorage, 142 r storage.ReadSeekCloser, 143 path string, 144 ) (int64, error) { 145 wrapper := &readerWrapper{ 146 ReadSeekCloser: r, 147 store: store, 148 ctx: ctx, 149 path: path, 150 } 151 var err error 152 res := new(preader.ParquetReader) 153 res.NP = 1 154 res.PFile = wrapper 155 if err = res.ReadFooter(); err != nil { 156 return 0, err 157 } 158 numRows := res.Footer.NumRows 159 if err = wrapper.Close(); err != nil { 160 return 0, err 161 } 162 return numRows, nil 163 } 164 165 func NewParquetParser( 166 ctx context.Context, 167 store storage.ExternalStorage, 168 r storage.ReadSeekCloser, 169 path string, 170 ) (*ParquetParser, error) { 171 // check to avoid wrapping twice 172 wrapper, ok := r.(source.ParquetFile) 173 if !ok { 174 wrapper = &readerWrapper{ 175 ReadSeekCloser: r, 176 store: store, 177 ctx: ctx, 178 path: path, 179 } 180 } 181 182 // FIXME: need to bench what the best value for the concurrent reader number 183 reader, err := preader.NewParquetReader(wrapper, nil, 2) 184 if err != nil { 185 return nil, errors.Trace(err) 186 } 187 188 columns := make([]string, 0, len(reader.Footer.Schema)-1) 189 columnMetas := make([]*parquet.SchemaElement, 0, len(reader.Footer.Schema)-1) 190 for _, c := range reader.SchemaHandler.SchemaElements { 191 if c.GetNumChildren() == 0 { 192 // NOTE: the SchemaElement.Name is capitalized, SchemaHandler.Infos.ExName is the raw column name 193 // though in this context, there is no difference between these two fields 194 columns = append(columns, strings.ToLower(c.Name)) 195 // transfer old ConvertedType to LogicalType 196 columnMeta := c 197 if c.ConvertedType != nil && c.LogicalType == nil { 198 newMeta := *c 199 columnMeta = &newMeta 200 if err := convertToLogicType(columnMeta); err != nil { 201 return nil, err 202 } 203 } 204 columnMetas = append(columnMetas, columnMeta) 205 } 206 } 207 208 return &ParquetParser{ 209 Reader: reader, 210 columns: columns, 211 columnMetas: columnMetas, 212 logger: log.L(), 213 }, nil 214 } 215 216 func convertToLogicType(se *parquet.SchemaElement) error { 217 logicalType := &parquet.LogicalType{} 218 switch *se.ConvertedType { 219 case parquet.ConvertedType_UTF8: 220 logicalType.STRING = &parquet.StringType{} 221 case parquet.ConvertedType_ENUM: 222 logicalType.ENUM = &parquet.EnumType{} 223 case parquet.ConvertedType_DECIMAL: 224 logicalType.DECIMAL = &parquet.DecimalType{ 225 Scale: *se.Scale, 226 Precision: *se.Precision, 227 } 228 case parquet.ConvertedType_DATE: 229 logicalType.DATE = &parquet.DateType{} 230 case parquet.ConvertedType_TIME_MILLIS: 231 logicalType.TIME = &parquet.TimeType{ 232 IsAdjustedToUTC: true, 233 Unit: &parquet.TimeUnit{ 234 MILLIS: parquet.NewMilliSeconds(), 235 }, 236 } 237 case parquet.ConvertedType_TIME_MICROS: 238 logicalType.TIME = &parquet.TimeType{ 239 IsAdjustedToUTC: true, 240 Unit: &parquet.TimeUnit{ 241 MICROS: parquet.NewMicroSeconds(), 242 }, 243 } 244 case parquet.ConvertedType_TIMESTAMP_MILLIS: 245 logicalType.TIMESTAMP = &parquet.TimestampType{ 246 IsAdjustedToUTC: true, 247 Unit: &parquet.TimeUnit{ 248 MILLIS: parquet.NewMilliSeconds(), 249 }, 250 } 251 case parquet.ConvertedType_TIMESTAMP_MICROS: 252 logicalType.TIMESTAMP = &parquet.TimestampType{ 253 IsAdjustedToUTC: true, 254 Unit: &parquet.TimeUnit{ 255 MICROS: parquet.NewMicroSeconds(), 256 }, 257 } 258 case parquet.ConvertedType_UINT_8: 259 logicalType.INTEGER = &parquet.IntType{ 260 BitWidth: 8, 261 IsSigned: false, 262 } 263 case parquet.ConvertedType_UINT_16: 264 logicalType.INTEGER = &parquet.IntType{ 265 BitWidth: 16, 266 IsSigned: false, 267 } 268 case parquet.ConvertedType_UINT_32: 269 logicalType.INTEGER = &parquet.IntType{ 270 BitWidth: 32, 271 IsSigned: false, 272 } 273 case parquet.ConvertedType_UINT_64: 274 logicalType.INTEGER = &parquet.IntType{ 275 BitWidth: 64, 276 IsSigned: false, 277 } 278 case parquet.ConvertedType_INT_8: 279 logicalType.INTEGER = &parquet.IntType{ 280 BitWidth: 8, 281 IsSigned: true, 282 } 283 case parquet.ConvertedType_INT_16: 284 logicalType.INTEGER = &parquet.IntType{ 285 BitWidth: 16, 286 IsSigned: true, 287 } 288 case parquet.ConvertedType_INT_32: 289 logicalType.INTEGER = &parquet.IntType{ 290 BitWidth: 32, 291 IsSigned: true, 292 } 293 case parquet.ConvertedType_INT_64: 294 logicalType.INTEGER = &parquet.IntType{ 295 BitWidth: 64, 296 IsSigned: true, 297 } 298 case parquet.ConvertedType_JSON: 299 logicalType.JSON = &parquet.JsonType{} 300 case parquet.ConvertedType_BSON: 301 logicalType.BSON = &parquet.BsonType{} 302 // case parquet.ConvertedType_INTERVAL, parquet.ConvertedType_MAP, parquet.ConvertedType_MAP_KEY_VALUE, parquet.ConvertedType_LIST: 303 default: 304 return errors.Errorf("unsupported type: '%s'", *se.ConvertedType) 305 } 306 se.LogicalType = logicalType 307 return nil 308 } 309 310 // Pos returns the currently row number of the parquet file 311 func (pp *ParquetParser) Pos() (pos int64, rowID int64) { 312 return pp.curStart + int64(pp.curIndex), pp.lastRow.RowID 313 } 314 315 func (pp *ParquetParser) SetPos(pos int64, rowID int64) error { 316 if pos < pp.curStart { 317 panic("don't support seek back yet") 318 } 319 pp.lastRow.RowID = rowID 320 321 if pos < pp.curStart+int64(len(pp.rows)) { 322 pp.curIndex = int(pos - pp.curStart) 323 pp.readRows = pos 324 return nil 325 } 326 327 if pos > pp.curStart+int64(len(pp.rows)) { 328 if err := pp.Reader.SkipRows(pos - pp.curStart - int64(len(pp.rows))); err != nil { 329 return errors.Trace(err) 330 } 331 } 332 pp.curStart = pos 333 pp.readRows = pos 334 pp.curIndex = 0 335 if len(pp.rows) > 0 { 336 pp.rows = pp.rows[:0] 337 } 338 339 return nil 340 } 341 342 func (pp *ParquetParser) Close() error { 343 pp.Reader.ReadStop() 344 return pp.Reader.PFile.Close() 345 } 346 347 func (pp *ParquetParser) ReadRow() error { 348 pp.lastRow.RowID++ 349 pp.lastRow.Length = 0 350 if pp.curIndex >= len(pp.rows) { 351 if pp.readRows >= pp.Reader.GetNumRows() { 352 return io.EOF 353 } 354 count := batchReadRowSize 355 if pp.Reader.GetNumRows()-pp.readRows < int64(count) { 356 count = int(pp.Reader.GetNumRows() - pp.readRows) 357 } 358 359 var err error 360 pp.rows, err = pp.Reader.ReadByNumber(count) 361 if err != nil { 362 return errors.Trace(err) 363 } 364 pp.curStart = pp.readRows 365 pp.readRows += int64(len(pp.rows)) 366 pp.curIndex = 0 367 } 368 369 row := pp.rows[pp.curIndex] 370 pp.curIndex++ 371 372 v := reflect.ValueOf(row) 373 length := v.NumField() 374 if cap(pp.lastRow.Row) < length { 375 pp.lastRow.Row = make([]types.Datum, length) 376 } else { 377 pp.lastRow.Row = pp.lastRow.Row[:length] 378 } 379 for i := 0; i < length; i++ { 380 pp.lastRow.Length += getDatumLen(v.Field(i)) 381 if err := setDatumValue(&pp.lastRow.Row[i], v.Field(i), pp.columnMetas[i]); err != nil { 382 return err 383 } 384 } 385 return nil 386 } 387 388 func getDatumLen(v reflect.Value) int { 389 if v.Kind() == reflect.Ptr { 390 if v.IsNil() { 391 return 0 392 } else { 393 return getDatumLen(v.Elem()) 394 } 395 } 396 if v.Kind() == reflect.String { 397 return len(v.String()) 398 } 399 return 8 400 } 401 402 // convert a parquet value to Datum 403 // 404 // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md 405 func setDatumValue(d *types.Datum, v reflect.Value, meta *parquet.SchemaElement) error { 406 switch v.Kind() { 407 case reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64: 408 d.SetUint64(v.Uint()) 409 case reflect.Int8, reflect.Int16: 410 d.SetInt64(v.Int()) 411 case reflect.Int32, reflect.Int64: 412 return setDatumByInt(d, v.Int(), meta) 413 case reflect.String: 414 setDatumByString(d, v.String(), meta) 415 case reflect.Float32, reflect.Float64: 416 d.SetFloat64(v.Float()) 417 case reflect.Ptr: 418 if v.IsNil() { 419 d.SetNull() 420 } else { 421 return setDatumValue(d, v.Elem(), meta) 422 } 423 default: 424 log.L().Error("unknown value", zap.Stringer("kind", v.Kind()), 425 zap.String("type", v.Type().Name()), zap.Reflect("value", v.Interface())) 426 return errors.Errorf("unknown value: %v", v) 427 } 428 return nil 429 } 430 431 func setDatumByString(d *types.Datum, v string, meta *parquet.SchemaElement) { 432 if meta.LogicalType != nil && meta.LogicalType.DECIMAL != nil { 433 v = binaryToDecimalStr([]byte(v), int(meta.LogicalType.DECIMAL.Scale)) 434 } 435 d.SetString(v, "") 436 } 437 438 func binaryToDecimalStr(rawBytes []byte, scale int) string { 439 negative := rawBytes[0] > 127 440 if negative { 441 for i := 0; i < len(rawBytes); i++ { 442 rawBytes[i] = ^rawBytes[i] 443 } 444 for i := len(rawBytes) - 1; i >= 0; i-- { 445 rawBytes[i] += 1 446 if rawBytes[i] != 0 { 447 break 448 } 449 } 450 } 451 452 intValue := big.NewInt(0) 453 intValue = intValue.SetBytes(rawBytes) 454 val := fmt.Sprintf("%0*d", scale, intValue) 455 dotIndex := len(val) - scale 456 var res strings.Builder 457 if negative { 458 res.WriteByte('-') 459 } 460 if dotIndex == 0 { 461 res.WriteByte('0') 462 } else { 463 res.WriteString(val[:dotIndex]) 464 } 465 if scale > 0 { 466 res.WriteByte('.') 467 res.WriteString(val[dotIndex:]) 468 } 469 return res.String() 470 } 471 472 // when the value type is int32/int64, convert to value to target logical type in tidb 473 func setDatumByInt(d *types.Datum, v int64, meta *parquet.SchemaElement) error { 474 if meta.ConvertedType == nil && meta.LogicalType == nil { 475 d.SetInt64(v) 476 return nil 477 } 478 479 logicalType := meta.LogicalType 480 switch { 481 case logicalType.DECIMAL != nil: 482 if logicalType.DECIMAL.Scale == 0 { 483 d.SetInt64(v) 484 return nil 485 } 486 minLen := logicalType.DECIMAL.Scale + 1 487 if v < 0 { 488 minLen++ 489 } 490 val := fmt.Sprintf("%0*d", minLen, v) 491 dotIndex := len(val) - int(*meta.Scale) 492 d.SetString(val[:dotIndex]+"."+val[dotIndex:], "") 493 case logicalType.DATE != nil: 494 dateStr := time.Unix(v*86400, 0).Format("2006-01-02") 495 d.SetString(dateStr, "") 496 case logicalType.TIMESTAMP != nil: 497 // convert all timestamp types (datetime/timestamp) to string 498 timeStr := formatTime(v, logicalType.TIMESTAMP.Unit, "2006-01-02 15:04:05.999999", 499 "2006-01-02 15:04:05.999999Z", logicalType.TIMESTAMP.IsAdjustedToUTC) 500 d.SetString(timeStr, "") 501 case logicalType.TIME != nil: 502 // convert all timestamp types (datetime/timestamp) to string 503 timeStr := formatTime(v, logicalType.TIME.Unit, "15:04:05.999999", "15:04:05.999999Z", 504 logicalType.TIME.IsAdjustedToUTC) 505 d.SetString(timeStr, "") 506 default: 507 d.SetInt64(v) 508 } 509 return nil 510 } 511 512 func formatTime(v int64, units *parquet.TimeUnit, format, utcFormat string, utc bool) string { 513 var sec, nsec int64 514 if units.MICROS != nil { 515 sec = v / 1e6 516 nsec = (v % 1e6) * 1e3 517 } else if units.MILLIS != nil { 518 sec = v / 1e3 519 nsec = (v % 1e3) * 1e6 520 } else { 521 // nano 522 sec = v / 1e9 523 nsec = v % 1e9 524 } 525 t := time.Unix(sec, nsec).UTC() 526 if utc { 527 return t.Format(utcFormat) 528 } 529 return t.Format(format) 530 } 531 532 func (pp *ParquetParser) LastRow() Row { 533 return pp.lastRow 534 } 535 536 func (pp *ParquetParser) RecycleRow(row Row) { 537 } 538 539 // Columns returns the _lower-case_ column names corresponding to values in 540 // the LastRow. 541 func (pp *ParquetParser) Columns() []string { 542 return pp.columns 543 } 544 545 // SetColumns set restored column names to parser 546 func (pp *ParquetParser) SetColumns(cols []string) { 547 // just do nothing 548 } 549 550 func (pp *ParquetParser) SetLogger(l log.Logger) { 551 pp.logger = l 552 }