diff --git a/distsql/distsql.go b/distsql/distsql.go index 77b75efc480fd..5581f9c269c99 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -133,6 +133,7 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie memTracker: kvReq.MemTracker, encodeType: encodetype, storeType: kvReq.StoreType, + paging: kvReq.Paging, }, nil } diff --git a/distsql/select_result.go b/distsql/select_result.go index 3ac7f1db94a97..b2eef6e6f5300 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -152,7 +152,8 @@ type selectResult struct { durationReported bool memTracker *memory.Tracker - stats *selectResultRuntimeStats + stats *selectResultRuntimeStats + paging bool } func (r *selectResult) fetchResp(ctx context.Context) error { @@ -206,7 +207,11 @@ func (r *selectResult) fetchResp(ctx context.Context) error { // final round of fetch // TODO: Add a label to distinguish between success or failure. // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + if r.paging { + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "paging").Observe(r.fetchDuration.Seconds()) + } else { + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "common").Observe(r.fetchDuration.Seconds()) + } r.durationReported = true } return nil diff --git a/distsql/stream.go b/distsql/stream.go index 73d8f96b8fe79..2f10e72b5c0f5 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -82,7 +82,7 @@ func (r *streamResult) readDataFromResponse(ctx context.Context, resp kv.Respons if !r.durationReported { // TODO: Add a label to distinguish between success or failure. // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "streaming").Observe(r.fetchDuration.Seconds()) r.durationReported = true } return true, nil diff --git a/executor/builder.go b/executor/builder.go index 78c94fd02f1aa..ef932da1211fa 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3409,6 +3409,11 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn if err != nil { return nil, err } + indexPaging := false + if v.Paging { + indexPaging = true + indexStreaming = false + } tableReq, tableStreaming, tbl, err := buildTableReq(b, v.Schema().Len(), v.TablePlans) if err != nil { return nil, err @@ -3430,6 +3435,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn columns: ts.Columns, indexStreaming: indexStreaming, tableStreaming: tableStreaming, + indexPaging: indexPaging, dataReaderBuilder: &dataReaderBuilder{executorBuilder: b}, corColInIdxSide: b.corColInDistPlan(v.IndexPlans), corColInTblSide: b.corColInDistPlan(v.TablePlans), diff --git a/executor/distsql.go b/executor/distsql.go index 1c31130dc53b5..9d380883703cd 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -361,6 +361,7 @@ type IndexLookUpExecutor struct { indexStreaming bool tableStreaming bool + indexPaging bool corColInIdxSide bool corColInTblSide bool @@ -560,6 +561,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< SetDesc(e.desc). SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). + SetPaging(e.indexPaging). SetReadReplicaScope(e.readReplicaScope). SetIsStaleness(e.isStaleness). SetFromSessionVars(e.ctx.GetSessionVars()). diff --git a/metrics/distsql.go b/metrics/distsql.go index 9bec9d7646827..3a4527da510ae 100644 --- a/metrics/distsql.go +++ b/metrics/distsql.go @@ -27,7 +27,7 @@ var ( Name: "handle_query_duration_seconds", Help: "Bucketed histogram of processing time (s) of handled queries.", Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }, []string{LblType, LblSQLType}) + }, []string{LblType, LblSQLType, LblCoprType}) DistSQLScanKeysPartialHistogram = prometheus.NewHistogram( prometheus.HistogramOpts{ diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index f6606d18ef4c6..a828967a8ada9 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -6168,10 +6168,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (copr_type)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "", + "legendFormat": "{{copr_type}}", "metric": "tidb_distsql_query_total", "refId": "A", "step": 4 diff --git a/metrics/session.go b/metrics/session.go index 0058104788f21..83df91439d311 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -142,6 +142,7 @@ const ( LblDb = "db" LblResult = "result" LblSQLType = "sql_type" + LblCoprType = "copr_type" LblGeneral = "general" LblInternal = "internal" LbTxnMode = "txn_mode" diff --git a/planner/core/explain.go b/planner/core/explain.go index 26bfa775fc417..d0ae474b79e7f 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -456,17 +456,22 @@ func (p *PhysicalIndexReader) accessObject(sctx sessionctx.Context) string { // ExplainInfo implements Plan interface. func (p *PhysicalIndexLookUpReader) ExplainInfo() string { + var str strings.Builder // The children can be inferred by the relation symbol. if p.PushedLimit != nil { - var str strings.Builder str.WriteString("limit embedded(offset:") str.WriteString(strconv.FormatUint(p.PushedLimit.Offset, 10)) str.WriteString(", count:") str.WriteString(strconv.FormatUint(p.PushedLimit.Count, 10)) str.WriteString(")") - return str.String() } - return "" + if p.Paging { + if p.PushedLimit != nil { + str.WriteString(", ") + } + str.WriteString("paging:true") + } + return str.String() } func (p *PhysicalIndexLookUpReader) accessObject(sctx sessionctx.Context) string { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 5157ffb738b24..ff90a92b9b497 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1249,6 +1249,7 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid indexPlan: is, tblColHists: ds.TblColHists, tblCols: ds.TblCols, + expectCnt: uint64(prop.ExpectedCnt), } cop.partitionInfo = PartitionInfo{ PruningConds: ds.allConds, diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 17e84b6efcf47..6293bba4b5073 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -270,6 +270,7 @@ type PhysicalIndexLookUpReader struct { TablePlans []PhysicalPlan indexPlan PhysicalPlan tablePlan PhysicalPlan + Paging bool ExtraHandleCol *expression.Column // PushedLimit is used to avoid unnecessary table scan tasks of IndexLookUpReader. diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 4437a354b1757..288444c9f2c12 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -18,6 +18,7 @@ import ( "bytes" "fmt" "strings" + "testing" . "github.com/pingcap/check" "github.com/pingcap/tidb/config" @@ -26,6 +27,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" + kit "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/testkit" @@ -639,3 +641,65 @@ func (s *testPlanNormalize) TestIssue25729(c *C) { tk.MustExec("insert into t1 values(\"a\", \"adwa\");") tk.MustQuery("select * from t1 where concat(a, b) like \"aadwa\" and a = \"a\";").Check(testkit.Rows("a adwa")) } + +func TestCopPaging(t *testing.T) { + store, clean := kit.CreateMockStore(t) + defer clean() + + tk := kit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set session tidb_enable_paging = 1") + tk.MustExec("create table t(id int, c1 int, c2 int, primary key (id), key i(c1))") + defer tk.MustExec("drop table t") + for i := 0; i < 1024; i++ { + tk.MustExec("insert into t values(?, ?, ?)", i, i, i) + } + tk.MustExec("analyze table t") + + // limit 960 should go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + "Limit 4.00 root offset:0, count:960", + "└─IndexLookUp 4.00 root paging:true", + " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 4.00 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 1024.00 cop[tikv] table:t keep order:false")) + } + + // selection between limit and indexlookup, limit 960 should also go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + "Limit 3.20 root offset:0, count:960", + "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", + " └─IndexLookUp 3.20 root paging:true", + " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + } + + // limit 961 exceeds the threshold, it should not go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + "Limit 4.00 root offset:0, count:961", + "└─IndexLookUp 4.00 root ", + " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 4.00 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 1024.00 cop[tikv] table:t keep order:false")) + } + + // selection between limit and indexlookup, limit 961 should not go paging too + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + "Limit 3.20 root offset:0, count:961", + "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", + " └─IndexLookUp 3.20 root ", + " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + } +} diff --git a/planner/core/task.go b/planner/core/task.go index 4b9c5692ca29d..90fbcedacaa9b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/paging" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -89,6 +90,10 @@ type copTask struct { // For table partition. partitionInfo PartitionInfo + + // expectCnt is the expected row count of upper task, 0 for unlimited. + // It's used for deciding whether using paging distsql. + expectCnt uint64 } func (t *copTask) invalid() bool { @@ -914,7 +919,17 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { // (indexRows / batchSize) * batchSize * CPUFactor // Since we don't know the number of copTasks built, ignore these network cost now. indexRows := t.indexPlan.statsInfo().RowCount - newTask.cst += indexRows * sessVars.CPUFactor + idxCst := indexRows * sessVars.CPUFactor + // if the expectCnt is below the paging threshold, using paging API, recalculate idxCst. + // paging API reduces the count of index and table rows, however introduces more seek cost. + if ctx.GetSessionVars().EnablePaging && t.expectCnt > 0 && t.expectCnt <= paging.Threshold { + p.Paging = true + pagingCst := calcPagingCost(ctx, t) + // prevent enlarging the cost because we take paging as a better plan, + // if the cost is enlarged, it'll be easier to go another plan. + idxCst = math.Min(idxCst, pagingCst) + } + newTask.cst += idxCst // Add cost of worker goroutines in index lookup. numTblWorkers := float64(sessVars.IndexLookupConcurrency()) newTask.cst += (numTblWorkers + 1) * sessVars.ConcurrencyFactor @@ -951,6 +966,41 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { return newTask } +func extractRows(p PhysicalPlan) float64 { + f := float64(0) + for _, c := range p.Children() { + if len(c.Children()) != 0 { + f += extractRows(c) + } else { + f += c.statsInfo().RowCount + } + } + return f +} + +// calcPagingCost calculates the cost for paging processing which may increase the seekCnt and reduce scanned rows. +func calcPagingCost(ctx sessionctx.Context, t *copTask) float64 { + sessVars := ctx.GetSessionVars() + indexRows := t.indexPlan.statsInfo().RowCount + expectCnt := t.expectCnt + sourceRows := extractRows(t.indexPlan) + // with paging, the scanned rows is always less than or equal to source rows. + if uint64(sourceRows) < expectCnt { + expectCnt = uint64(sourceRows) + } + seekCnt := paging.CalculateSeekCnt(expectCnt) + indexSelectivity := float64(1) + if sourceRows > indexRows { + indexSelectivity = indexRows / sourceRows + } + pagingCst := seekCnt*sessVars.GetSeekFactor(nil) + float64(expectCnt)*sessVars.CPUFactor + pagingCst *= indexSelectivity + + // we want the diff between idxCst and pagingCst here, + // however, the idxCst does not contain seekFactor, so a seekFactor needs to be removed + return pagingCst - sessVars.GetSeekFactor(nil) +} + func (t *rootTask) convertToRootTask(_ sessionctx.Context) *rootTask { return t.copy().(*rootTask) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 25ed965ea15e3..04e29a8507656 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/paging" "github.com/pingcap/tidb/util/trxevents" "github.com/pingcap/tipb/go-tipb" "github.com/tikv/client-go/v2/metrics" @@ -61,18 +62,6 @@ const ( copNextMaxBackoff = 20000 ) -// A paging request may be separated into multi requests if there are more data than a page. -// The paging size grows from min to max, it's not well tuned yet. -// e.g. a paging request scans over range (r1, r200), it requires 64 rows in the first batch, -// if it's not drained, then the paging size grows, the new range is calculated like (r100, r200), then send a request again. -// Compare with the common unary request, paging request allows early access of data, it offers a streaming-like way processing data. -// TODO: may make the paging parameters configurable. -const ( - minPagingSize uint64 = 64 - maxPagingSize = minPagingSize * 128 - pagingSizeGrow uint64 = 2 -) - // CopClient is coprocessor client. type CopClient struct { kv.RequestTypeSupportedChecker @@ -212,7 +201,7 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv // the size will grow every round. pagingSize := uint64(0) if req.Paging { - pagingSize = minPagingSize + pagingSize = paging.MinPagingSize } tasks = append(tasks, &copTask{ region: loc.Location.Region, @@ -928,7 +917,7 @@ func (worker *copIteratorWorker) handleCopPagingResult(bo *Backoffer, rpcCtx *ti if task.ranges.Len() == 0 { return nil, nil } - task.pagingSize = growPagingSize(task.pagingSize) + task.pagingSize = paging.GrowPagingSize(task.pagingSize) return []*copTask{task}, nil } @@ -1332,11 +1321,3 @@ func isolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { return kvrpcpb.IsolationLevel_SI } } - -func growPagingSize(size uint64) uint64 { - size *= pagingSizeGrow - if size > maxPagingSize { - return maxPagingSize - } - return size -} diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index 88ad5568f68eb..b628b4eaab831 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver/backoff" + "github.com/pingcap/tidb/util/paging" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" @@ -318,7 +319,7 @@ func TestBuildPagingTasks(t *testing.T) { require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], "a", "c") require.True(t, tasks[0].paging) - require.Equal(t, tasks[0].pagingSize, minPagingSize) + require.Equal(t, tasks[0].pagingSize, paging.MinPagingSize) } func toCopRange(r kv.KeyRange) *coprocessor.KeyRange { diff --git a/util/paging/main_test.go b/util/paging/main_test.go new file mode 100644 index 0000000000000..af568af279474 --- /dev/null +++ b/util/paging/main_test.go @@ -0,0 +1,27 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package paging + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + goleak.VerifyTestMain(m) +} diff --git a/util/paging/paging.go b/util/paging/paging.go new file mode 100644 index 0000000000000..5f2618ea341db --- /dev/null +++ b/util/paging/paging.go @@ -0,0 +1,61 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package paging + +import "math" + +// A paging request may be separated into multi requests if there are more data than a page. +// The paging size grows from min to max, it's not well tuned yet. +// e.g. a paging request scans over range (r1, r200), it requires 64 rows in the first batch, +// if it's not drained, then the paging size grows, the new range is calculated like (r100, r200), then send a request again. +// Compare with the common unary request, paging request allows early access of data, it offers a streaming-like way processing data. +// TODO: may make the paging parameters configurable. +const ( + MinPagingSize uint64 = 64 + maxPagingSizeShift = 7 + pagingSizeGrow = 2 + MaxPagingSize = MinPagingSize << maxPagingSizeShift + pagingGrowingSum = ((2 << maxPagingSizeShift) - 1) * MinPagingSize + Threshold uint64 = 960 +) + +// GrowPagingSize grows the paging size and ensures it does not exceed MaxPagingSize +func GrowPagingSize(size uint64) uint64 { + size <<= 1 + if size > MaxPagingSize { + return MaxPagingSize + } + return size +} + +// CalculateSeekCnt calculates the seek count from expect count +func CalculateSeekCnt(expectCnt uint64) float64 { + if expectCnt == 0 { + return 0 + } + if expectCnt > pagingGrowingSum { + // if the expectCnt is larger than pagingGrowingSum, calculate the seekCnt for the excess. + return float64(8 + (expectCnt-pagingGrowingSum+MaxPagingSize-1)/MaxPagingSize) + } + if expectCnt > MinPagingSize { + // if the expectCnt is less than pagingGrowingSum, + // calculate the seekCnt(number of terms) from the sum of a geometric progression. + // expectCnt = minPagingSize * (pagingSizeGrow ^ seekCnt - 1) / (pagingSizeGrow - 1) + // simplify (pagingSizeGrow ^ seekCnt - 1) to pagingSizeGrow ^ seekCnt, we can infer that + // seekCnt = log((pagingSizeGrow - 1) * expectCnt / minPagingSize) / log(pagingSizeGrow) + return 1 + float64(int(math.Log(float64((pagingSizeGrow-1)*expectCnt)/float64(MinPagingSize))/math.Log(float64(pagingSizeGrow)))) + } + return 1 +} diff --git a/util/paging/paging_test.go b/util/paging/paging_test.go new file mode 100644 index 0000000000000..1890b4d754d54 --- /dev/null +++ b/util/paging/paging_test.go @@ -0,0 +1,36 @@ +// Copyright 2021 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package paging + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestGrowPagingSize(t *testing.T) { + require.Equal(t, GrowPagingSize(MinPagingSize), MinPagingSize*pagingSizeGrow) + require.Equal(t, GrowPagingSize(MaxPagingSize), MaxPagingSize) + require.Equal(t, GrowPagingSize(MaxPagingSize/pagingSizeGrow+1), MaxPagingSize) +} + +func TestCalculateSeekCnt(t *testing.T) { + require.InDelta(t, CalculateSeekCnt(0), 0, 0.1) + require.InDelta(t, CalculateSeekCnt(1), 1, 0.1) + require.InDelta(t, CalculateSeekCnt(MinPagingSize), 1, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum), maxPagingSizeShift+1, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum+1), maxPagingSizeShift+2, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum+MaxPagingSize), maxPagingSizeShift+2, 0.1) +}