90 Star 490 Fork 149

平凯星辰(北京)科技有限公司/tidb

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
克隆/下载
distsql.go 2.88 KB
一键复制 编辑 原始数据 按行查看 历史
// Copyright 2017 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 distsql
import (
"github.com/juju/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"golang.org/x/net/context"
)
// XAPI error codes.
const (
codeInvalidResp = 1
)
// Select sends a DAG request, returns SelectResult.
// In kvReq, KeyRanges is required, Concurrency/KeepOrder/Desc/IsolationLevel/Priority are optional.
func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fieldTypes []*types.FieldType, fb *statistics.QueryFeedback) (SelectResult, error) {
// For testing purpose.
if hook := ctx.Value("CheckSelectRequestHook"); hook != nil {
hook.(func(*kv.Request))(kvReq)
}
if !sctx.GetSessionVars().EnableStreaming {
kvReq.Streaming = false
}
resp := sctx.GetClient().Send(ctx, kvReq, sctx.GetSessionVars().KVVars)
if resp == nil {
err := errors.New("client returns nil response")
return nil, errors.Trace(err)
}
if kvReq.Streaming {
return &streamResult{
resp: resp,
rowLen: len(fieldTypes),
fieldTypes: fieldTypes,
ctx: sctx,
feedback: fb,
}, nil
}
return &selectResult{
label: "dag",
resp: resp,
results: make(chan resultWithErr, kvReq.Concurrency),
closed: make(chan struct{}),
rowLen: len(fieldTypes),
fieldTypes: fieldTypes,
ctx: sctx,
feedback: fb,
}, nil
}
// Analyze do a analyze request.
func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv.Variables) (SelectResult, error) {
resp := client.Send(ctx, kvReq, vars)
if resp == nil {
return nil, errors.New("client returns nil response")
}
result := &selectResult{
label: "analyze",
resp: resp,
results: make(chan resultWithErr, kvReq.Concurrency),
closed: make(chan struct{}),
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
}
return result, nil
}
// Checksum sends a checksum request.
func Checksum(ctx context.Context, client kv.Client, kvReq *kv.Request, vars *kv.Variables) (SelectResult, error) {
resp := client.Send(ctx, kvReq, vars)
if resp == nil {
return nil, errors.New("client returns nil response")
}
result := &selectResult{
label: "checksum",
resp: resp,
results: make(chan resultWithErr, kvReq.Concurrency),
closed: make(chan struct{}),
feedback: statistics.NewQueryFeedback(0, nil, 0, false),
}
return result, nil
}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/pingcap/tidb.git
git@gitee.com:pingcap/tidb.git
pingcap
tidb
tidb
v2.1.0-rc.1

搜索帮助