From 0beac1800c7fb81fab98d70282dc26de66537f1a Mon Sep 17 00:00:00 2001 From: Liqi Geng Date: Sat, 2 Apr 2022 14:32:29 +0800 Subject: [PATCH 1/2] expression: fix the wrong rounding behavior of Decimal (#33278) close pingcap/tidb#32213 --- executor/aggfuncs/func_avg.go | 4 +- executor/aggfuncs/func_first_row.go | 2 +- executor/aggfuncs/func_max_min.go | 2 +- executor/aggfuncs/func_sum.go | 2 +- executor/insert_test.go | 19 +++++++++ expression/aggregation/avg.go | 2 +- expression/builtin_arithmetic.go | 2 +- expression/builtin_arithmetic_vec.go | 2 +- expression/builtin_cast.go | 2 +- expression/builtin_cast_vec.go | 2 +- expression/builtin_math.go | 4 +- expression/builtin_math_vec.go | 4 +- expression/builtin_time.go | 8 ++-- expression/builtin_time_vec.go | 4 +- expression/constant.go | 2 +- types/convert_test.go | 2 +- types/datum.go | 47 +++++++++++++--------- types/datum_test.go | 58 ++++++++++++++++++++++++++++ types/mydecimal.go | 26 ++++++------- types/mydecimal_benchmark_test.go | 4 +- types/mydecimal_test.go | 4 +- util/codec/codec.go | 2 +- util/rowcodec/decoder.go | 2 +- 23 files changed, 147 insertions(+), 59 deletions(-) diff --git a/executor/aggfuncs/func_avg.go b/executor/aggfuncs/func_avg.go index 17203aad50b7e..15713eedab880 100644 --- a/executor/aggfuncs/func_avg.go +++ b/executor/aggfuncs/func_avg.go @@ -81,7 +81,7 @@ func (e *baseAvgDecimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr Par if frac == -1 { frac = mysql.MaxDecimalScale } - err = finalResult.Round(finalResult, frac, types.ModeHalfEven) + err = finalResult.Round(finalResult, frac, types.ModeHalfUp) if err != nil { return err } @@ -276,7 +276,7 @@ func (e *avgOriginal4DistinctDecimal) AppendFinalResult2Chunk(sctx sessionctx.Co if frac == -1 { frac = mysql.MaxDecimalScale } - err = finalResult.Round(finalResult, frac, types.ModeHalfEven) + err = finalResult.Round(finalResult, frac, types.ModeHalfUp) if err != nil { return err } diff --git a/executor/aggfuncs/func_first_row.go b/executor/aggfuncs/func_first_row.go index d93724d0eda4b..88f9d2bf75c5e 100644 --- a/executor/aggfuncs/func_first_row.go +++ b/executor/aggfuncs/func_first_row.go @@ -485,7 +485,7 @@ func (e *firstRow4Decimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr P if frac == -1 { frac = mysql.MaxDecimalScale } - err := p.val.Round(&p.val, frac, types.ModeHalfEven) + err := p.val.Round(&p.val, frac, types.ModeHalfUp) if err != nil { return err } diff --git a/executor/aggfuncs/func_max_min.go b/executor/aggfuncs/func_max_min.go index 956e32649eb50..73e49264472fb 100644 --- a/executor/aggfuncs/func_max_min.go +++ b/executor/aggfuncs/func_max_min.go @@ -822,7 +822,7 @@ func (e *maxMin4Decimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr Par if frac == -1 { frac = mysql.MaxDecimalScale } - err := p.val.Round(&p.val, frac, types.ModeHalfEven) + err := p.val.Round(&p.val, frac, types.ModeHalfUp) if err != nil { return err } diff --git a/executor/aggfuncs/func_sum.go b/executor/aggfuncs/func_sum.go index 1d0bf624172cb..0e153a51d98d2 100644 --- a/executor/aggfuncs/func_sum.go +++ b/executor/aggfuncs/func_sum.go @@ -178,7 +178,7 @@ func (e *sum4Decimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr Partia if frac == -1 { frac = mysql.MaxDecimalScale } - err := p.val.Round(&p.val, frac, types.ModeHalfEven) + err := p.val.Round(&p.val, frac, types.ModeHalfUp) if err != nil { return err } diff --git a/executor/insert_test.go b/executor/insert_test.go index ab35060fefc17..ce91449f5fe49 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1984,3 +1984,22 @@ func TestInsertIntoSelectError(t *testing.T) { tk.MustQuery("SELECT * FROM t1;").Check(testkit.Rows("0", "0", "0")) tk.MustExec("DROP TABLE t1;") } + +// https://github.com/pingcap/tidb/issues/32213. +func TestIssue32213(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + + tk.MustExec("create table test.t1(c1 float)") + tk.MustExec("insert into test.t1 values(999.99)") + tk.MustQuery("select cast(test.t1.c1 as decimal(4, 1)) from test.t1").Check(testkit.Rows("999.9")) + tk.MustQuery("select cast(test.t1.c1 as decimal(5, 1)) from test.t1").Check(testkit.Rows("1000.0")) + + tk.MustExec("drop table if exists test.t1") + tk.MustExec("create table test.t1(c1 decimal(6, 4))") + tk.MustExec("insert into test.t1 values(99.9999)") + tk.MustQuery("select cast(test.t1.c1 as decimal(5, 3)) from test.t1").Check(testkit.Rows("99.999")) + tk.MustQuery("select cast(test.t1.c1 as decimal(6, 3)) from test.t1").Check(testkit.Rows("100.000")) +} diff --git a/expression/aggregation/avg.go b/expression/aggregation/avg.go index 9dd4b508cc79c..c1e32c92c847f 100644 --- a/expression/aggregation/avg.go +++ b/expression/aggregation/avg.go @@ -86,7 +86,7 @@ func (af *avgFunction) GetResult(evalCtx *AggEvaluateContext) (d types.Datum) { if frac == -1 { frac = mysql.MaxDecimalScale } - err = to.Round(to, mathutil.Min(frac, mysql.MaxDecimalScale), types.ModeHalfEven) + err = to.Round(to, mathutil.Min(frac, mysql.MaxDecimalScale), types.ModeHalfUp) terror.Log(err) d.SetMysqlDecimal(to) } diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index 7291889a24812..4bb4927ff1835 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -721,7 +721,7 @@ func (s *builtinArithmeticDivideDecimalSig) evalDecimal(row chunk.Row) (*types.M } else if err == nil { _, frac := c.PrecisionAndFrac() if frac < s.baseBuiltinFunc.tp.Decimal { - err = c.Round(c, s.baseBuiltinFunc.tp.Decimal, types.ModeHalfEven) + err = c.Round(c, s.baseBuiltinFunc.tp.Decimal, types.ModeHalfUp) } } else if err == types.ErrOverflow { err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s / %s)", s.args[0].String(), s.args[1].String())) diff --git a/expression/builtin_arithmetic_vec.go b/expression/builtin_arithmetic_vec.go index 07c0965ffe669..42e076b803dc1 100644 --- a/expression/builtin_arithmetic_vec.go +++ b/expression/builtin_arithmetic_vec.go @@ -100,7 +100,7 @@ func (b *builtinArithmeticDivideDecimalSig) vecEvalDecimal(input *chunk.Chunk, r } else if err == nil { _, frac = to.PrecisionAndFrac() if frac < b.baseBuiltinFunc.tp.Decimal { - if err = to.Round(&to, b.baseBuiltinFunc.tp.Decimal, types.ModeHalfEven); err != nil { + if err = to.Round(&to, b.baseBuiltinFunc.tp.Decimal, types.ModeHalfUp); err != nil { return err } } diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index 8ec15676860ee..a3c8442773565 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -969,7 +969,7 @@ func (b *builtinCastDecimalAsIntSig) evalInt(row chunk.Row) (res int64, isNull b // Round is needed for both unsigned and signed. var to types.MyDecimal - err = val.Round(&to, 0, types.ModeHalfEven) + err = val.Round(&to, 0, types.ModeHalfUp) if err != nil { return 0, true, err } diff --git a/expression/builtin_cast_vec.go b/expression/builtin_cast_vec.go index f07b4f6d8d4a0..dbaf5c27d3ddf 100644 --- a/expression/builtin_cast_vec.go +++ b/expression/builtin_cast_vec.go @@ -1743,7 +1743,7 @@ func (b *builtinCastDecimalAsIntSig) vecEvalInt(input *chunk.Chunk, result *chun // Round is needed for both unsigned and signed. to := d64s[i] - err = d64s[i].Round(&to, 0, types.ModeHalfEven) + err = d64s[i].Round(&to, 0, types.ModeHalfUp) if err != nil { return err } diff --git a/expression/builtin_math.go b/expression/builtin_math.go index 54fae95639142..ee17b4e490ffe 100644 --- a/expression/builtin_math.go +++ b/expression/builtin_math.go @@ -393,7 +393,7 @@ func (b *builtinRoundDecSig) evalDecimal(row chunk.Row) (*types.MyDecimal, bool, return nil, isNull, err } to := new(types.MyDecimal) - if err = val.Round(to, 0, types.ModeHalfEven); err != nil { + if err = val.Round(to, 0, types.ModeHalfUp); err != nil { return nil, true, err } return to, false, nil @@ -469,7 +469,7 @@ func (b *builtinRoundWithFracDecSig) evalDecimal(row chunk.Row) (*types.MyDecima return nil, isNull, err } to := new(types.MyDecimal) - if err = val.Round(to, mathutil.Min(int(frac), b.tp.Decimal), types.ModeHalfEven); err != nil { + if err = val.Round(to, mathutil.Min(int(frac), b.tp.Decimal), types.ModeHalfUp); err != nil { return nil, true, err } return to, false, nil diff --git a/expression/builtin_math_vec.go b/expression/builtin_math_vec.go index 13b6a701d5d8e..c0dfa90b870dd 100644 --- a/expression/builtin_math_vec.go +++ b/expression/builtin_math_vec.go @@ -392,7 +392,7 @@ func (b *builtinRoundDecSig) vecEvalDecimal(input *chunk.Chunk, result *chunk.Co if result.IsNull(i) { continue } - if err := d64s[i].Round(buf, 0, types.ModeHalfEven); err != nil { + if err := d64s[i].Round(buf, 0, types.ModeHalfUp); err != nil { return err } d64s[i] = *buf @@ -994,7 +994,7 @@ func (b *builtinRoundWithFracDecSig) vecEvalDecimal(input *chunk.Chunk, result * continue } // TODO: reuse d64[i] and remove the temporary variable tmp. - if err := d64s[i].Round(tmp, mathutil.Min(int(i64s[i]), b.tp.Decimal), types.ModeHalfEven); err != nil { + if err := d64s[i].Round(tmp, mathutil.Min(int(i64s[i]), b.tp.Decimal), types.ModeHalfUp); err != nil { return err } d64s[i] = *tmp diff --git a/expression/builtin_time.go b/expression/builtin_time.go index d9cdcb9504e4b..23862fe9f4276 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -1734,7 +1734,7 @@ func evalFromUnixTime(ctx sessionctx.Context, fsp int, unixTimeStamp *types.MyDe sc := ctx.GetSessionVars().StmtCtx tmp := time.Unix(integralPart, fractionalPart).In(sc.TimeZone) - t, err := convertTimeToMysqlTime(tmp, fsp, types.ModeHalfEven) + t, err := convertTimeToMysqlTime(tmp, fsp, types.ModeHalfUp) if err != nil { return res, true, err } @@ -2050,7 +2050,7 @@ func (b *builtinSysDateWithFspSig) evalTime(row chunk.Row) (d types.Time, isNull loc := b.ctx.GetSessionVars().Location() now := time.Now().In(loc) - result, err := convertTimeToMysqlTime(now, int(fsp), types.ModeHalfEven) + result, err := convertTimeToMysqlTime(now, int(fsp), types.ModeHalfUp) if err != nil { return types.ZeroTime, true, err } @@ -2072,7 +2072,7 @@ func (b *builtinSysDateWithoutFspSig) Clone() builtinFunc { func (b *builtinSysDateWithoutFspSig) evalTime(row chunk.Row) (d types.Time, isNull bool, err error) { tz := b.ctx.GetSessionVars().Location() now := time.Now().In(tz) - result, err := convertTimeToMysqlTime(now, 0, types.ModeHalfEven) + result, err := convertTimeToMysqlTime(now, 0, types.ModeHalfUp) if err != nil { return types.ZeroTime, true, err } @@ -2393,7 +2393,7 @@ func evalUTCTimestampWithFsp(ctx sessionctx.Context, fsp int) (types.Time, bool, if err != nil { return types.ZeroTime, true, err } - result, err := convertTimeToMysqlTime(nowTs.UTC(), fsp, types.ModeHalfEven) + result, err := convertTimeToMysqlTime(nowTs.UTC(), fsp, types.ModeHalfUp) if err != nil { return types.ZeroTime, true, err } diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index 47ede27eea6ee..11531945c551d 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -172,7 +172,7 @@ func (b *builtinSysDateWithoutFspSig) vecEvalTime(input *chunk.Chunk, result *ch result.ResizeTime(n, false) times := result.Times() - t, err := convertTimeToMysqlTime(now, 0, types.ModeHalfEven) + t, err := convertTimeToMysqlTime(now, 0, types.ModeHalfUp) if err != nil { return err } @@ -775,7 +775,7 @@ func (b *builtinSysDateWithFspSig) vecEvalTime(input *chunk.Chunk, result *chunk if result.IsNull(i) { continue } - t, err := convertTimeToMysqlTime(now, int(ds[i]), types.ModeHalfEven) + t, err := convertTimeToMysqlTime(now, int(ds[i]), types.ModeHalfUp) if err != nil { return err } diff --git a/expression/constant.go b/expression/constant.go index e77bd5aea39a0..31f3733f146c0 100644 --- a/expression/constant.go +++ b/expression/constant.go @@ -295,7 +295,7 @@ func (c *Constant) EvalDecimal(ctx sessionctx.Context, row chunk.Row) (*types.My // The decimal may be modified during plan building. _, frac := res.PrecisionAndFrac() if frac < c.GetType().Decimal { - err = res.Round(res, c.GetType().Decimal, types.ModeHalfEven) + err = res.Round(res, c.GetType().Decimal, types.ModeHalfUp) } return res, false, err } diff --git a/types/convert_test.go b/types/convert_test.go index 0c15297c3db43..6a4f91a5d30ff 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -841,7 +841,7 @@ func TestConvert(t *testing.T) { dec := NewDecFromInt(-123) err := dec.Shift(-5) require.NoError(t, err) - err = dec.Round(dec, 5, ModeHalfEven) + err = dec.Round(dec, 5, ModeHalfUp) require.NoError(t, err) signedAccept(t, mysql.TypeNewDecimal, dec, "-0.00123") } diff --git a/types/datum.go b/types/datum.go index 76c47b2c1e5ab..ce3ed2ee23076 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1170,7 +1170,7 @@ func (d *Datum) convertToUint(sc *stmtctx.StatementContext, target *FieldType) ( } case KindMysqlTime: dec := d.GetMysqlTime().ToNumber() - err = dec.Round(dec, 0, ModeHalfEven) + err = dec.Round(dec, 0, ModeHalfUp) ival, err1 := dec.ToInt() if err == nil { err = err1 @@ -1181,7 +1181,7 @@ func (d *Datum) convertToUint(sc *stmtctx.StatementContext, target *FieldType) ( } case KindMysqlDuration: dec := d.GetMysqlDuration().ToNumber() - err = dec.Round(dec, 0, ModeHalfEven) + err = dec.Round(dec, 0, ModeHalfUp) ival, err1 := dec.ToInt() if err1 == nil { val, err = ConvertIntToUint(sc, ival, upperBound, tp) @@ -1436,8 +1436,11 @@ func (d *Datum) convertToMysqlDecimal(sc *stmtctx.StatementContext, target *Fiel default: return invalidConv(d, target.Tp) } - var err1 error - dec, err1 = ProduceDecWithSpecifiedTp(dec, target, sc) + dec1, err1 := ProduceDecWithSpecifiedTp(dec, target, sc) + // If there is a error, dec1 may be nil. + if dec1 != nil { + dec = dec1 + } if err == nil && err1 != nil { err = err1 } @@ -1458,21 +1461,29 @@ func ProduceDecWithSpecifiedTp(dec *MyDecimal, tp *FieldType, sc *stmtctx.Statem if flen < decimal { return nil, ErrMBiggerThanD.GenWithStackByArgs("") } - prec, frac := dec.PrecisionAndFrac() - if !dec.IsZero() && prec-frac > flen-decimal { + + var old *MyDecimal + if int(dec.digitsFrac) > decimal { + old = new(MyDecimal) + *old = *dec + } + if int(dec.digitsFrac) != decimal { + // Error doesn't matter because the following code will check the new decimal + // and set error if any. + _ = dec.Round(dec, decimal, ModeHalfUp) + } + + _, digitsInt := dec.removeLeadingZeros() + // After rounding decimal, the new decimal may have a longer integer length which may be longer than expected. + // So the check of integer length must be after rounding. + // E.g. "99.9999", flen 5, decimal 3, Round("99.9999", 3, ModelHalfUp) -> "100.000". + if flen-decimal < digitsInt { + // Integer length is longer, choose the max or min decimal. dec = NewMaxOrMinDec(dec.IsNegative(), flen, decimal) - // select (cast 111 as decimal(1)) causes a warning in MySQL. + // select cast(111 as decimal(1)) causes a warning in MySQL. err = ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%d, %d)", flen, decimal)) - } else if frac != decimal { - old := *dec - err = dec.Round(dec, decimal, ModeHalfEven) - if err != nil { - return nil, err - } - if !old.IsZero() && frac > decimal && dec.Compare(&old) != 0 { - sc.AppendWarning(ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", &old)) - err = nil - } + } else if old != nil && dec.Compare(old) != 0 { + sc.AppendWarning(ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", old)) } } @@ -1835,7 +1846,7 @@ func (d *Datum) toSignedInteger(sc *stmtctx.StatementContext, tp byte) (int64, e return ival, errors.Trace(err) case KindMysqlDecimal: var to MyDecimal - err := d.GetMysqlDecimal().Round(&to, 0, ModeHalfEven) + err := d.GetMysqlDecimal().Round(&to, 0, ModeHalfUp) ival, err1 := to.ToInt() if err == nil { err = err1 diff --git a/types/datum_test.go b/types/datum_test.go index 063f1703f9537..90985f3cf8cfe 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/hack" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -566,3 +567,60 @@ func BenchmarkCompareDatumByReflect(b *testing.B) { reflect.DeepEqual(vals, vals1) } } + +func TestProduceDecWithSpecifiedTp(t *testing.T) { + tests := []struct { + dec string + flen int + frac int + newDec string + isOverflow bool + isTruncated bool + }{ + {"0.0000", 4, 3, "0.000", false, false}, + {"0.0001", 4, 3, "0.000", false, true}, + {"123", 8, 5, "123.00000", false, false}, + {"-123", 8, 5, "-123.00000", false, false}, + {"123.899", 5, 2, "123.90", false, true}, + {"-123.899", 5, 2, "-123.90", false, true}, + {"123.899", 6, 2, "123.90", false, true}, + {"-123.899", 6, 2, "-123.90", false, true}, + {"123.99", 4, 1, "124.0", false, true}, + {"123.99", 3, 0, "124", false, true}, + {"-123.99", 3, 0, "-124", false, true}, + {"123.99", 3, 1, "99.9", true, false}, + {"-123.99", 3, 1, "-99.9", true, false}, + {"99.9999", 5, 3, "99.999", true, false}, + {"-99.9999", 5, 3, "-99.999", true, false}, + {"99.9999", 6, 3, "100.000", false, true}, + {"-99.9999", 6, 3, "-100.000", false, true}, + } + sc := new(stmtctx.StatementContext) + for _, tt := range tests { + tp := &FieldType{ + Tp: mysql.TypeNewDecimal, + Flen: tt.flen, + Decimal: tt.frac, + } + dec := NewDecFromStringForTest(tt.dec) + newDec, err := ProduceDecWithSpecifiedTp(dec, tp, sc) + if tt.isOverflow { + if !ErrOverflow.Equal(err) { + assert.FailNow(t, "Error is not overflow", "err: %v before: %v after: %v", err, tt.dec, dec) + } + } else { + require.NoError(t, err, tt) + } + require.Equal(t, tt.newDec, newDec.String()) + warn := sc.TruncateWarnings(0) + if tt.isTruncated { + if len(warn) != 1 || !ErrTruncatedWrongVal.Equal(warn[0].Err) { + assert.FailNow(t, "Warn is not truncated", "warn: %v before: %v after: %v", warn, tt.dec, dec) + } + } else { + if warn != nil { + assert.FailNow(t, "Warn is not nil", "warn: %v before: %v after: %v", warn, tt.dec, dec) + } + } + } +} diff --git a/types/mydecimal.go b/types/mydecimal.go index e0cbe3a928aed..c1e09808c6d20 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -52,12 +52,12 @@ const ( DivFracIncr = 4 - // ModeHalfEven rounds normally. - ModeHalfEven RoundMode = 5 + // Round up to the next integer if positive or down to the next integer if negative. + ModeHalfUp RoundMode = 5 // Truncate just truncates the decimal. ModeTruncate RoundMode = 10 // Ceiling is not supported now. - modeCeiling RoundMode = 0 + ModeCeiling RoundMode = 0 pow10off int = 81 ) @@ -265,7 +265,7 @@ func (d *MyDecimal) GetDigitsInt() int8 { // String returns the decimal string representation rounded to resultFrac. func (d *MyDecimal) String() string { tmp := *d - err := tmp.Round(&tmp, int(tmp.resultFrac), ModeHalfEven) + err := tmp.Round(&tmp, int(tmp.resultFrac), ModeHalfUp) terror.Log(errors.Trace(err)) return string(tmp.ToString()) } @@ -586,7 +586,7 @@ func (d *MyDecimal) Shift(shift int) error { err = ErrTruncated wordsFrac -= lack diff := digitsFrac - wordsFrac*digitsPerWord - err1 := d.Round(d, digitEnd-point-diff, ModeHalfEven) + err1 := d.Round(d, digitEnd-point-diff, ModeHalfUp) if err1 != nil { return errors.Trace(err1) } @@ -802,15 +802,15 @@ func (d *MyDecimal) doMiniRightShift(shift, beg, end int) { // to - result buffer. d == to is allowed // frac - to what position after fraction point to round. can be negative! // roundMode - round to nearest even or truncate -// ModeHalfEven rounds normally. -// Truncate just truncates the decimal. +// ModeHalfUp rounds normally. +// ModeTruncate just truncates the decimal. // // NOTES -// scale can be negative ! +// frac can be negative ! // one TRUNCATED error (line XXX below) isn't treated very logical :( // // RETURN VALUE -// eDecOK/eDecTruncated +// nil/ErrTruncated/ErrOverflow func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err error) { // wordsFracTo is the number of fraction words in buffer. wordsFracTo := (frac + 1) / digitsPerWord @@ -860,7 +860,7 @@ func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err err doInc := false switch roundMode { // Notice: No support for ceiling mode now. - case modeCeiling: + case ModeCeiling: // If any word after scale is not zero, do increment. // e.g ceiling 3.0001 to scale 1, gets 3.1 idx := toIdx + (wordsFrac - wordsFracTo) @@ -871,10 +871,10 @@ func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err err } idx-- } - case ModeHalfEven: + case ModeHalfUp: digAfterScale := d.wordBuf[toIdx+1] / digMask // the first digit after scale. - // If first digit after scale is 5 and round even, do increment if digit at scale is odd. - doInc = (digAfterScale > 5) || (digAfterScale == 5) + // If first digit after scale is equal to or greater than 5, do increment. + doInc = digAfterScale >= 5 case ModeTruncate: // Never round, just truncate. doInc = false diff --git a/types/mydecimal_benchmark_test.go b/types/mydecimal_benchmark_test.go index e6065b41a4045..aec99db606735 100644 --- a/types/mydecimal_benchmark_test.go +++ b/types/mydecimal_benchmark_test.go @@ -79,7 +79,7 @@ func BenchmarkRound(b *testing.B) { b.StartTimer() for n := 0; n < b.N; n++ { for i := 0; i < len(tests); i++ { - err := tests[i].inputDec.Round(&roundTo, tests[i].scale, ModeHalfEven) + err := tests[i].inputDec.Round(&roundTo, tests[i].scale, ModeHalfUp) if err != nil { b.Fatal(err) } @@ -91,7 +91,7 @@ func BenchmarkRound(b *testing.B) { } } for i := 0; i < len(tests); i++ { - err := tests[i].inputDec.Round(&roundTo, tests[i].scale, modeCeiling) + err := tests[i].inputDec.Round(&roundTo, tests[i].scale, ModeCeiling) if err != nil { b.Fatal(err) } diff --git a/types/mydecimal_test.go b/types/mydecimal_test.go index 13da9284ad395..07e8df28c28b9 100644 --- a/types/mydecimal_test.go +++ b/types/mydecimal_test.go @@ -322,7 +322,7 @@ func TestRoundWithHalfEven(t *testing.T) { err := dec.FromString([]byte(ca.input)) require.NoError(t, err) var rounded MyDecimal - err = dec.Round(&rounded, ca.scale, ModeHalfEven) + err = dec.Round(&rounded, ca.scale, ModeHalfUp) require.Equal(t, ca.err, err) result := rounded.ToString() require.Equal(t, ca.output, string(result)) @@ -393,7 +393,7 @@ func TestRoundWithCeil(t *testing.T) { err := dec.FromString([]byte(ca.input)) require.NoError(t, err) var rounded MyDecimal - err = dec.Round(&rounded, ca.scale, modeCeiling) + err = dec.Round(&rounded, ca.scale, ModeCeiling) require.Equal(t, ca.err, err) result := rounded.ToString() require.Equal(t, ca.output, string(result)) diff --git a/util/codec/codec.go b/util/codec/codec.go index 28be507f2a826..f3f6043dbe524 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -1107,7 +1107,7 @@ func (decoder *Decoder) DecodeOne(b []byte, colIdx int, ft *types.FieldType) (re } if ft.Decimal != types.UnspecifiedLength && frac > ft.Decimal { to := new(types.MyDecimal) - err := dec.Round(to, ft.Decimal, types.ModeHalfEven) + err := dec.Round(to, ft.Decimal, types.ModeHalfUp) if err != nil { return nil, errors.Trace(err) } diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index 6bccac22214be..fd0fbd8c6aa9d 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -300,7 +300,7 @@ func (decoder *ChunkDecoder) decodeColToChunk(colIdx int, col *ColInfo, colData } if col.Ft.Decimal != types.UnspecifiedLength && frac > col.Ft.Decimal { to := new(types.MyDecimal) - err := dec.Round(to, col.Ft.Decimal, types.ModeHalfEven) + err := dec.Round(to, col.Ft.Decimal, types.ModeHalfUp) if err != nil { return errors.Trace(err) } From 9b358bc73912cfbcbbf031f3390b0046f853177b Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 2 Apr 2022 15:34:29 +0800 Subject: [PATCH 2/2] gomod: update ristretto and sarama to fix data race (#33652) close pingcap/tidb#33649, ref pingcap/tidb#33667 --- go.mod | 9 ++++++--- go.sum | 33 ++++++++++++++++----------------- 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/go.mod b/go.mod index b44762a2e571b..e67a9405e2617 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/BurntSushi/toml v0.3.1 github.com/DATA-DOG/go-sqlmock v1.5.0 github.com/Jeffail/gabs/v2 v2.5.1 - github.com/Shopify/sarama v1.32.0 + github.com/Shopify/sarama v1.29.0 github.com/aws/aws-sdk-go v1.35.3 github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d github.com/carlmjohnson/flagext v0.21.0 @@ -21,7 +21,7 @@ require ( github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 - github.com/dgraph-io/ristretto v0.1.0 + github.com/dgraph-io/ristretto v0.1.1-0.20211108053508-297c39e6640f github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 github.com/docker/go-units v0.4.0 github.com/fsouza/fake-gcs-server v1.19.0 @@ -83,7 +83,7 @@ require ( golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8 golang.org/x/sync v0.0.0-20210220032951-036812b2e83c - golang.org/x/sys v0.0.0-20220310020820-b874c991c1a5 + golang.org/x/sys v0.0.0-20220330033206-e17cdc41300f golang.org/x/text v0.3.7 golang.org/x/time v0.0.0-20220224211638-0e9765cccd65 golang.org/x/tools v0.1.8 @@ -205,3 +205,6 @@ replace github.com/pingcap/tidb/parser => ./parser // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible + +// it can be removed after merging https://github.com/dgraph-io/ristretto/pull/294 +replace github.com/dgraph-io/ristretto => github.com/hawkingrei/ristretto v0.1.1-0.20220402052934-7556ec01f9db diff --git a/go.sum b/go.sum index 8106de35c6732..4c967837d4479 100644 --- a/go.sum +++ b/go.sum @@ -81,10 +81,10 @@ github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKz github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= -github.com/Shopify/sarama v1.32.0 h1:P+RUjEaRU0GMMbYexGMDyrMkLhbbBVUVISDywi+IlFU= -github.com/Shopify/sarama v1.32.0/go.mod h1:+EmJJKZWVT/faR9RcOxJerP+LId4iWdQPBGLy1Y1Njs= -github.com/Shopify/toxiproxy/v2 v2.3.0 h1:62YkpiP4bzdhKMH+6uC5E95y608k3zDwdzuBMsnn3uQ= -github.com/Shopify/toxiproxy/v2 v2.3.0/go.mod h1:KvQTtB6RjCJY4zqNJn7C7JDFgsG5uoHYDirfUfpIm0c= +github.com/Shopify/sarama v1.29.0 h1:ARid8o8oieau9XrHI55f/L3EoRAhm9px6sonbD7yuUE= +github.com/Shopify/sarama v1.29.0/go.mod h1:2QpgD79wpdAESqNQMxNc0KYMkycd4slxGdV3TWSVqrU= +github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= +github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= github.com/VividCortex/ewma v1.1.1 h1:MnEK4VOv6n0RSY4vtRe3h11qjxL3+t0B8yOL8iMXdcM= github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmxzcbUokwA= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= @@ -180,7 +180,6 @@ github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzA github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.1/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= @@ -196,8 +195,6 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/ristretto v0.1.0 h1:Jv3CGQHp9OjuMBSne1485aDpUkTKEcUqF+jm/LuerPI= -github.com/dgraph-io/ristretto v0.1.0/go.mod h1:fux0lOrBhrVCJd3lcTHsIJhq1T2rokOu6v9Vcb3Q9ug= github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= @@ -244,8 +241,8 @@ github.com/form3tech-oss/jwt-go v3.2.5+incompatible/go.mod h1:pbq4aXjuKjdthFRnoD github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= -github.com/frankban/quicktest v1.14.2 h1:SPb1KFFmM+ybpEjPUhCCkZOM5xlovT5UbrMvWnXyBns= -github.com/frankban/quicktest v1.14.2/go.mod h1:mgiwOwqx65TmIk1wJ6Q7wvnVMocbUorkibMOrVTHZps= +github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= +github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.5.1 h1:mZcQUHVQUQWoPXXtuf9yuEXKudkV2sx1E06UadKWpgI= @@ -437,6 +434,8 @@ github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= +github.com/hawkingrei/ristretto v0.1.1-0.20220402052934-7556ec01f9db h1:TgwwlrryS+4yDB95DS7DNHt62YuLcMfUQ0+k42lHiao= +github.com/hawkingrei/ristretto v0.1.1-0.20220402052934-7556ec01f9db/go.mod h1:RAy2GVV4sTWVlNMavv3xhLsk18rxhfhDnombTe6EF5c= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.2.0 h1:05I4QRnGpI0m37iZQRuskXh+w77mr6Z41lwQzuHLwW0= @@ -506,7 +505,7 @@ github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0 github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.10.5/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/klauspost/compress v1.11.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/compress v1.14.4/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= +github.com/klauspost/compress v1.12.2/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.15.1 h1:y9FcTHGyrebwfP0ZZqFiaxTaiDnUrGkJkI+f583BL1A= github.com/klauspost/compress v1.15.1/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= @@ -609,6 +608,7 @@ github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwp github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= +github.com/pierrec/lz4 v2.6.0+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pingcap/badger v1.5.1-0.20220314162537-ab58fbf40580 h1:MKVFZuqFvAMiDtv3AbihOQ6rY5IE8LWflI1BuZ/hF0Y= @@ -768,7 +768,6 @@ github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVK github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= -github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= @@ -776,9 +775,8 @@ github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPU github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f h1:9DDCDwOyEy/gId+IEMrFHLuQ5R/WV0KNxWLler8X2OY= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sdOQnirw1PrcnTJYkmW1iOHtUmblMmGdUOHyWYycLI= -github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= -github.com/xdg-go/scram v1.1.0/go.mod h1:1WAq6h33pAW+iRreB34OORO2Nf7qel3VV3fjBj+hCSs= -github.com/xdg-go/stringprep v1.0.2/go.mod h1:8F9zXuvzgwmyT5DUm4GUfZGDdT3W+LCvS6+da4O5kxM= +github.com/xdg/scram v1.0.3/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= +github.com/xdg/stringprep v1.0.3/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= @@ -895,6 +893,7 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20220214200702-86341886e292 h1:f+lwQ+GtmgoY+A2YaQxlSOnDjXcQ7ZRLWOHbC6HtRqE= golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -987,12 +986,12 @@ golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd h1:O7DYs+zxREGLKzKoMQrtrEacpb0ZVXA5rIwylE2Xchk= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -1107,8 +1106,8 @@ golang.org/x/sys v0.0.0-20211210111614-af8b64212486/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220209214540-3681064d5158/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220310020820-b874c991c1a5 h1:y/woIyUBFbpQGKS0u1aHF/40WUDnek3fPOyD08H5Vng= -golang.org/x/sys v0.0.0-20220310020820-b874c991c1a5/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220330033206-e17cdc41300f h1:rlezHXNlxYWvBCzNses9Dlc7nGFaNMJeqLolcmQSSZY= +golang.org/x/sys v0.0.0-20220330033206-e17cdc41300f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=