代码拉取完成,页面将自动刷新
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package main
import (
"encoding/json"
"io/ioutil"
"math/rand"
"time"
"github.com/pingcap/tidb/model"
stats "github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/mock"
"github.com/pkg/errors"
log "github.com/sirupsen/logrus"
)
func loadStats(tblInfo *model.TableInfo, path string) (*stats.Table, error) {
data, err := ioutil.ReadFile(path)
if err != nil {
return nil, errors.Trace(err)
}
jsTable := &stats.JSONTable{}
err = json.Unmarshal(data, jsTable)
if err != nil {
return nil, errors.Trace(err)
}
handle := stats.NewHandle(mock.NewContext(), 0)
return handle.LoadStatsFromJSONToTable(tblInfo, jsTable)
}
type histogram struct {
stats.Histogram
index *model.IndexInfo
avgLen int
}
// When the randCnt falls in the middle of bucket, we return the idx of lower bound which is an even number.
// When the randCnt falls in the end of bucket, we return the upper bound which is odd.
func (h *histogram) getRandomBoundIdx() int {
cnt := h.Buckets[len(h.Buckets)-1].Count
randCnt := randInt64(0, cnt)
for i, bkt := range h.Buckets {
if bkt.Count >= randCnt {
if bkt.Count-bkt.Repeat > randCnt {
return 2 * i
}
return 2*i + 1
}
}
return 0
}
func (h *histogram) decodeInt(row *chunk.Row) int64 {
if h.index == nil {
return row.GetInt64(0)
}
data := row.GetBytes(0)
_, result, err := codec.DecodeInt(data)
if err != nil {
log.Fatal(err)
}
return result
}
func (h *histogram) randInt() int64 {
idx := h.getRandomBoundIdx()
if idx%2 == 0 {
lower := h.Bounds.GetRow(idx).GetInt64(0)
upper := h.Bounds.GetRow(idx + 1).GetInt64(0)
return randInt64(lower, upper)
}
return h.Bounds.GetRow(idx).GetInt64(0)
}
func (h *histogram) randFloat64() float64 {
idx := h.getRandomBoundIdx()
if idx%2 == 0 {
lower := h.Bounds.GetRow(idx).GetFloat64(0)
upper := h.Bounds.GetRow(idx + 1).GetFloat64(0)
rd := rand.Float64()
return lower + rd*(upper-lower)
}
return h.Bounds.GetRow(idx).GetFloat64(0)
}
func (h *histogram) randFloat32() float32 {
idx := h.getRandomBoundIdx()
if idx%2 == 0 {
lower := h.Bounds.GetRow(idx).GetFloat32(0)
upper := h.Bounds.GetRow(idx + 1).GetFloat32(0)
rd := rand.Float32()
return lower + rd*(upper-lower)
}
return h.Bounds.GetRow(idx).GetFloat32(0)
}
func (h *histogram) randDecimal() *types.MyDecimal {
idx := h.getRandomBoundIdx()
if idx%2 == 0 {
lower := h.Bounds.GetRow(idx).GetMyDecimal(0)
upper := h.Bounds.GetRow(idx + 1).GetMyDecimal(0)
rd := rand.Float64()
l, err := lower.ToFloat64()
if err != nil {
log.Fatal(err)
}
r, err := upper.ToFloat64()
if err != nil {
log.Fatal(err)
}
dec := &types.MyDecimal{}
err = dec.FromFloat64(l + rd*(r-l))
if err != nil {
log.Fatal(err)
}
return dec
}
return h.Bounds.GetRow(idx).GetMyDecimal(0)
}
func getValidPrefix(lower, upper string) string {
for i := range lower {
if i >= len(upper) {
log.Fatalf("lower %s is larger than upper %s", lower, upper)
}
if lower[i] != upper[i] {
randCh := uint8(rand.Intn(int(upper[i]-lower[i]))) + lower[i]
newBytes := make([]byte, i, i+1)
copy(newBytes, lower[:i])
newBytes = append(newBytes, byte(randCh))
return string(newBytes)
}
}
return lower
}
func (h *histogram) getAvgLen(maxLen int) int {
l := h.Bounds.NumRows()
totalLen := 0
for i := 0; i < l; i++ {
totalLen += len(h.Bounds.GetRow(i).GetString(0))
}
avg := totalLen / l
if avg > maxLen {
avg = maxLen
}
if avg == 0 {
avg = 1
}
return avg
}
func (h *histogram) randString() string {
idx := h.getRandomBoundIdx()
if idx%2 == 0 {
lower := h.Bounds.GetRow(idx).GetString(0)
upper := h.Bounds.GetRow(idx + 1).GetString(0)
prefix := getValidPrefix(lower, upper)
restLen := h.avgLen - len(prefix)
if restLen > 0 {
prefix = prefix + randString(restLen)
}
return prefix
}
return h.Bounds.GetRow(idx).GetString(0)
}
// randDate randoms a bucket and random a date between upper and lower bound.
func (h *histogram) randDate(unit string, mysqlFmt string, dateFmt string) string {
idx := h.getRandomBoundIdx()
if idx%2 == 0 {
lower := h.Bounds.GetRow(idx).GetTime(0)
upper := h.Bounds.GetRow(idx + 1).GetTime(0)
diff := types.TimestampDiff(unit, lower, upper)
if diff == 0 {
str, err := lower.DateFormat(mysqlFmt)
if err != nil {
log.Fatal(err)
}
return str
}
delta := randInt(0, int(diff)-1)
l, err := lower.Time.GoTime(time.Local)
if err != nil {
log.Fatal(err)
}
l = l.AddDate(0, 0, delta)
return l.Format(dateFmt)
}
str, err := h.Bounds.GetRow(idx).GetTime(0).DateFormat(mysqlFmt)
if err != nil {
log.Fatal(err)
}
return str
}
此处可能存在不合适展示的内容,页面不予展示。您可通过相关编辑功能自查并修改。
如您确认内容无涉及 不当用语 / 纯广告导流 / 暴力 / 低俗色情 / 侵权 / 盗版 / 虚假 / 无价值内容或违法国家有关法律法规的内容,可点击提交进行申诉,我们将尽快为您处理。