Skip to content
This repository has been archived by the owner on Jul 24, 2024. It is now read-only.

Commit

Permalink
resolve comments
Browse files Browse the repository at this point in the history
  • Loading branch information
glorv committed Jun 23, 2021
1 parent a2204cf commit 205d0b1
Show file tree
Hide file tree
Showing 2 changed files with 10 additions and 4 deletions.
12 changes: 8 additions & 4 deletions pkg/lightning/mydump/parquet_parser.go
Original file line number Diff line number Diff line change
Expand Up @@ -229,24 +229,28 @@ func convertToLogicType(se *parquet.SchemaElement) error {
logicalType.DATE = &parquet.DateType{}
case parquet.ConvertedType_TIME_MILLIS:
logicalType.TIME = &parquet.TimeType{
IsAdjustedToUTC: true,
Unit: &parquet.TimeUnit{
MILLIS: parquet.NewMilliSeconds(),
},
}
case parquet.ConvertedType_TIME_MICROS:
logicalType.TIME = &parquet.TimeType{
IsAdjustedToUTC: true,
Unit: &parquet.TimeUnit{
MICROS: parquet.NewMicroSeconds(),
},
}
case parquet.ConvertedType_TIMESTAMP_MILLIS:
logicalType.TIMESTAMP = &parquet.TimestampType{
IsAdjustedToUTC: true,
Unit: &parquet.TimeUnit{
MILLIS: parquet.NewMilliSeconds(),
},
}
case parquet.ConvertedType_TIMESTAMP_MICROS:
logicalType.TIMESTAMP = &parquet.TimestampType{
IsAdjustedToUTC: true,
Unit: &parquet.TimeUnit{
MICROS: parquet.NewMicroSeconds(),
},
Expand Down Expand Up @@ -401,7 +405,7 @@ func setDatumValue(d *types.Datum, v reflect.Value, meta *parquet.SchemaElement)
return setDatumValue(d, v.Elem(), meta)
}
default:
log.L().Fatal("unknown value", zap.Stringer("kind", v.Kind()),
log.L().Error("unknown value", zap.Stringer("kind", v.Kind()),
zap.String("type", v.Type().Name()), zap.Reflect("value", v.Interface()))
return errors.Errorf("unknown value: %v", v)
}
Expand Down Expand Up @@ -440,11 +444,11 @@ func binaryToDecimalStr(rawBytes []byte, scale int) string {
if dotIndex == 0 {
res.WriteByte('0')
} else {
res.Write([]byte(val[:dotIndex]))
res.WriteString(val[:dotIndex])
}
if scale > 0 {
res.WriteByte('.')
res.Write([]byte(val[dotIndex:]))
res.WriteString(val[dotIndex:])
}
return res.String()
}
Expand Down Expand Up @@ -488,7 +492,7 @@ func setDatumByInt(d *types.Datum, v int64, meta *parquet.SchemaElement) error {
sec = v / 1e9
nsec = v % 1e9
}
// TODO: how to deal with TimeZone
// TODO: how to deal with TimeZone if `IsAdjustedToUTC = false`
dateStr := time.Unix(sec, nsec).Format("2006-01-02 15:04:05.999")
d.SetString(dateStr, "")
case logicalType.TIME != nil:
Expand Down
2 changes: 2 additions & 0 deletions pkg/lightning/mydump/parquet_parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,6 +257,8 @@ func (s testParquetParserSuite) TestParquetAurora(c *C) {
c.Assert(v, Equals, row[j].GetInt64())
case string:
c.Assert(v, Equals, row[j].GetString())
default:
c.Error("unexpected value: ", expectedValues[j])
}
}
}
Expand Down

0 comments on commit 205d0b1

Please sign in to comment.