Skip to content

Commit

Permalink
Merge master to 2.0 to release rc5 (#6288)
Browse files Browse the repository at this point in the history
  • Loading branch information
shenli authored and zhexuany committed Apr 16, 2018
1 parent 83ea321 commit 217031a
Show file tree
Hide file tree
Showing 206 changed files with 14,875 additions and 4,661 deletions.
2 changes: 1 addition & 1 deletion Gopkg.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

18 changes: 12 additions & 6 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -71,13 +71,9 @@ parserlib: parser/parser.go
parser/parser.go: parser/parser.y
make parser

check: errcheck
go get github.com/golang/lint/golint
check: fmt errcheck lint vet

@echo "vet"
@ go tool vet -all -shadow $(TOPDIRS) 2>&1 | awk '{print} END{if(NR>0) {exit 1}}'
@echo "golint"
@ golint -set_exit_status $(PACKAGES)
fmt:
@echo "gofmt (simplify)"
@ gofmt -s -l -w $(FILES) 2>&1 | grep -v "vendor|parser/parser.go" | awk '{print} END{if(NR>0) {exit 1}}'

Expand All @@ -88,8 +84,18 @@ goword:

errcheck:
go get github.com/kisielk/errcheck
@echo "errcheck"
@ GOPATH=$(GOPATH) errcheck -blank $(PACKAGES) | grep -v "_test\.go" | awk '{print} END{if(NR>0) {exit 1}}'

lint:
go get github.com/golang/lint/golint
@echo "golint"
@ golint -set_exit_status $(PACKAGES)

vet:
@echo "vet"
@ go tool vet -all -shadow $(TOPDIRS) 2>&1 | awk '{print} END{if(NR>0) {exit 1}}'

clean:
$(GO) clean -i ./...
rm -rf *.out
Expand Down
24 changes: 12 additions & 12 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -8,31 +8,31 @@

## What is TiDB?

TiDB (The pronunciation is: /'taɪdiːbi:/ tai-D-B, etymology: titanium) is a Hybrid Transactional/Analytical Processing (HTAP) database. Inspired by the design of Google F1 and Google Spanner, TiDB features infinite horizontal scalability, strong consistency, and high availability. The goal of TiDB is to serve as a one-stop solution for online transactions and analyses.
TiDB (The pronunciation is: /'taɪdiːbi:/ tai-D-B, etymology: titanium) is an open source distributed scalable Hybrid Transactional and Analytical Processing (HTAP) database built by PingCAP. Inspired by the design of Google F1 and Google Spanner, TiDB features infinite horizontal scalability, strong consistency, and high availability. The goal of TiDB is to serve as a one-stop solution for both OLTP (Online Transactional Processing) and OLAP (Online Analytical Processing).

- __Horizontal scalability__

Grow TiDB as your business grows. You can increase the capacity for storage and computation simply by adding more machines.
TiDB provides horizontal scalability simply by adding new nodes. Never worry about infrastructure capacity ever again.

- __Compatible with MySQL protocol__
- __MySQL compatibility__

Use TiDB as MySQL. You can replace MySQL with TiDB to power your application without changing a single line of code in most cases.
Easily replace MySQL with TiDB to power your applications without changing a single line of code in most cases and still benefit from the MySQL ecosystem.

- __Automatic Failover and high availability__
- __Distributed transaction__

Your data and applications are always-on. TiDB automatically handles malfunctions and protects your applications from machine failures or even downtime of an entire data-center.
TiDB is your source of truth, guaranteeing ACID compliance, so your data is accurate and reliable anytime, anywhere.

- __Consistent distributed transactions__
- __Cloud Native__

Think of TiDB as a single-machine RDBMS. You can start a transaction that crosses multiple machines without worrying about consistency. TiDB makes your application code simple and robust.
TiDB is designed to work in the cloud -- public, private, or hybrid -- making deployment, provisioning, and maintenance drop-dead simple.

- __Online DDL__
- __No more ETL__

Evolve TiDB schemas as your requirement changes. You can add new columns and indexes without stopping or affecting the on-going operations.
ETL (Extract, Transform and Load) is no longer necessary with TiDB's hybrid OLTP/OLAP architecture, enabling you to create new values for your users, easier and faster.

- __Multiple storage engine support__
- __High availability__

Power TiDB with your most favorite engines. TiDB supports local storage engines such as GolevelDB, as well as [TiKV](https://github.com/pingcap/tikv), a distributed storage engine.
With TiDB, your data and applications are always on and continuously available, so your users are never disappointed.

For more details, see [How we build TiDB](https://pingcap.github.io/blog/2016/10/17/how-we-build-tidb/).

Expand Down
4 changes: 2 additions & 2 deletions ast/ast.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,8 +137,8 @@ type RecordSet interface {
// Fields gets result fields.
Fields() []*ResultField

// NextChunk reads records into chunk.
NextChunk(ctx context.Context, chk *chunk.Chunk) error
// Next reads records into chunk.
Next(ctx context.Context, chk *chunk.Chunk) error

// NewChunk creates a new chunk with initial capacity.
NewChunk() *chunk.Chunk
Expand Down
18 changes: 18 additions & 0 deletions ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,6 +403,7 @@ type CreateTableStmt struct {
Cols []*ColumnDef
Constraints []*Constraint
Options []*TableOption
Partition *PartitionOptions
}

// Accept implements Node Accept interface.
Expand Down Expand Up @@ -745,6 +746,7 @@ type AlterTableSpec struct {
OldColumnName *ColumnName
Position *ColumnPosition
LockType LockType
Comment string
}

// Accept implements Node Accept interface.
Expand Down Expand Up @@ -845,3 +847,19 @@ func (n *TruncateTableStmt) Accept(v Visitor) (Node, bool) {
n.Table = node.(*TableName)
return v.Leave(n)
}

// PartitionDefinition defines a single partition.
type PartitionDefinition struct {
Name string
LessThan []ExprNode
MaxValue bool
Comment string
}

// PartitionOptions specifies the partition options.
type PartitionOptions struct {
Tp model.PartitionType
Expr ExprNode
ColumnNames []*ColumnName
Definitions []*PartitionDefinition
}
1 change: 1 addition & 0 deletions ast/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -591,6 +591,7 @@ const (
AdminCancelDDLJobs
AdminCheckIndex
AdminRecoverIndex
AdminCleanupIndex
AdminCheckIndexRange
AdminShowDDLJobQueries
AdminChecksumTable
Expand Down
2 changes: 1 addition & 1 deletion cmd/benchdb/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ func (ut *benchDB) mustExec(sql string) {
rs := rss[0]
chk := rs.NewChunk()
for {
err := rs.NextChunk(ctx, chk)
err := rs.Next(ctx, chk)
if err != nil {
log.Fatal(err)
}
Expand Down
8 changes: 5 additions & 3 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,12 +144,13 @@ type Performance struct {
MaxProcs uint `toml:"max-procs" json:"max-procs"`
TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"`
RetryLimit uint `toml:"retry-limit" json:"retry-limit"`
JoinConcurrency uint `toml:"join-concurrency" json:"join-concurrency"`
CrossJoin bool `toml:"cross-join" json:"cross-join"`
StatsLease string `toml:"stats-lease" json:"stats-lease"`
RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"`
StmtCountLimit uint `toml:"stmt-count-limit" json:"stmt-count-limit"`
FeedbackProbability float64 `toml:"feedback-probability" json:"feedback-probability"`
QueryFeedbackLimit uint `toml:"query-feedback-limit" json:"query-feedback-limit"`
PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"`
}

// XProtocol is the XProtocol section of the config.
Expand Down Expand Up @@ -226,7 +227,7 @@ var defaultConf = Config{
Path: "/tmp/tidb",
RunDDL: true,
SplitTable: true,
Lease: "10s",
Lease: "45s",
TokenLimit: 1000,
OOMAction: "log",
EnableStreaming: false,
Expand All @@ -250,12 +251,13 @@ var defaultConf = Config{
Performance: Performance{
TCPKeepAlive: true,
RetryLimit: 10,
JoinConcurrency: 5,
CrossJoin: true,
StatsLease: "3s",
RunAutoAnalyze: true,
StmtCountLimit: 5000,
FeedbackProbability: 0,
QueryFeedbackLimit: 1024,
PseudoEstimateRatio: 0.7,
},
XProtocol: XProtocol{
XHost: "",
Expand Down
15 changes: 8 additions & 7 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ binlog-socket = ""
run-ddl = true

# Schema lease duration, very dangerous to change only if you know what you do.
lease = "10s"
lease = "45s"

# When create table, split a separated region for it. It is recommended to
# turn off this option if there will be a large number of tables created.
Expand All @@ -31,9 +31,6 @@ split-table = true
# The limit of concurrent executed sessions.
token-limit = 1000

# Enable chunk executors.
enable-chunk = true

# Only print a log when out of memory quota.
# Valid options: ["log", "cancel"]
oom-action = "log"
Expand Down Expand Up @@ -127,9 +124,6 @@ tcp-keep-alive = true
# The maximum number of retries when commit a transaction.
retry-limit = 10

# The number of goroutines that participate joining.
join-concurrency = 5

# Whether support cartesian product.
cross-join = true

Expand All @@ -142,6 +136,13 @@ run-auto-analyze = true
# Probability to use the query feedback to update stats, 0 or 1 for always false/true.
feedback-probability = 0.0

# The max number of query feedback that cache in memory.
query-feedback-limit = 1024

# Pseudo stats will be used if the ratio between the modify count and
# row count in statistics of a table is greater than it.
pseudo-estimate-ratio = 0.7

[proxy-protocol]
# PROXY protocol acceptable client networks.
# Empty string means disable PROXY protocol, * means all networks.
Expand Down
15 changes: 9 additions & 6 deletions ddl/callback_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,20 +16,24 @@ package ddl
import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/util/testleak"
"golang.org/x/net/context"
)

type TestDDLCallback struct {
*BaseCallback

onJobRunBefore func(*model.Job)
onJobUpdated func(*model.Job)
OnJobUpdatedExported func(*model.Job)
onWatched func(ctx context.Context)
onJobRunBefore func(*model.Job)
OnJobRunBeforeExported func(*model.Job)
onJobUpdated func(*model.Job)
OnJobUpdatedExported func(*model.Job)
onWatched func(ctx context.Context)
}

func (tc *TestDDLCallback) OnJobRunBefore(job *model.Job) {
if tc.OnJobRunBeforeExported != nil {
tc.OnJobRunBeforeExported(job)
return
}
if tc.onJobRunBefore != nil {
tc.onJobRunBefore(job)
return
Expand Down Expand Up @@ -61,7 +65,6 @@ func (tc *TestDDLCallback) OnWatched(ctx context.Context) {
}

func (s *testDDLSuite) TestCallback(c *C) {
defer testleak.AfterTest(c)()
cb := &BaseCallback{}
c.Assert(cb.OnChanged(nil), IsNil)
cb.OnJobRunBefore(nil)
Expand Down
19 changes: 15 additions & 4 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -440,7 +440,7 @@ func (d *ddl) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ error)
}

// doModifyColumn updates the column information and reorders all columns.
func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition) (ver int64, _ error) {
func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition) (ver int64, _ error) {
tblInfo, err := getTableInfo(t, job, job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
Expand All @@ -451,7 +451,18 @@ func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo
job.State = model.JobStateCancelled
return ver, infoschema.ErrColumnNotExists.GenByArgs(oldName, tblInfo.Name)
}
// If we want to rename the column name, we need to check whether it already exists.
if newCol.Name.L != oldName.L {
c := findCol(tblInfo.Columns, newCol.Name.L)
if c != nil {
job.State = model.JobStateCancelled
return ver, infoschema.ErrColumnExists.GenByArgs(newCol.Name)
}
}

// We need the latest column's offset and state. This information can be obtained from the store.
newCol.Offset = oldCol.Offset
newCol.State = oldCol.State
// Calculate column's new position.
oldPos, newPos := oldCol.Offset, oldCol.Offset
if pos.Tp == ast.ColumnPositionAfter {
Expand All @@ -477,10 +488,10 @@ func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo
}

columnChanged := make(map[string]*model.ColumnInfo)
columnChanged[oldName.L] = col
columnChanged[oldName.L] = newCol

if newPos == oldPos {
tblInfo.Columns[newPos] = col
tblInfo.Columns[newPos] = newCol
} else {
cols := tblInfo.Columns

Expand All @@ -490,7 +501,7 @@ func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo
} else {
copy(cols[oldPos:], cols[oldPos+1:newPos+1])
}
cols[newPos] = col
cols[newPos] = newCol

for i, col := range tblInfo.Columns {
if col.Offset != i {
Expand Down
7 changes: 6 additions & 1 deletion ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ type testColumnChangeSuite struct {
}

func (s *testColumnChangeSuite) SetUpSuite(c *C) {
testleak.BeforeTest()
s.store = testCreateStore(c, "test_column_change")
s.dbInfo = &model.DBInfo{
Name: model.NewCIStr("test_column_change"),
Expand All @@ -53,8 +54,12 @@ func (s *testColumnChangeSuite) SetUpSuite(c *C) {
c.Check(err, IsNil)
}

func (s *testColumnChangeSuite) TearDownSuite(c *C) {
s.store.Close()
testleak.AfterTest(c)()
}

func (s *testColumnChangeSuite) TestColumnChange(c *C) {
defer testleak.AfterTest(c)()
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
defer d.Stop()
// create table t (c1 int, c2 int);
Expand Down
5 changes: 2 additions & 3 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ type testColumnSuite struct {
}

func (s *testColumnSuite) SetUpSuite(c *C) {
testleak.BeforeTest()
s.store = testCreateStore(c, "test_column")
s.d = testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)

Expand All @@ -57,6 +58,7 @@ func (s *testColumnSuite) TearDownSuite(c *C) {

err := s.store.Close()
c.Assert(err, IsNil)
testleak.AfterTest(c)()
}

func testCreateColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo,
Expand Down Expand Up @@ -105,7 +107,6 @@ func testDropColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo,
}

func (s *testColumnSuite) TestColumn(c *C) {
defer testleak.AfterTest(c)()
tblInfo := testTableInfo(c, s.d, "t1", 3)
ctx := testNewContext(s.d)

Expand Down Expand Up @@ -736,7 +737,6 @@ func (s *testColumnSuite) testGetColumn(t table.Table, name string, isExist bool
}

func (s *testColumnSuite) TestAddColumn(c *C) {
defer testleak.AfterTest(c)()
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
tblInfo := testTableInfo(c, d, "t", 3)
ctx := testNewContext(d)
Expand Down Expand Up @@ -822,7 +822,6 @@ func (s *testColumnSuite) TestAddColumn(c *C) {
}

func (s *testColumnSuite) TestDropColumn(c *C) {
defer testleak.AfterTest(c)()
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
tblInfo := testTableInfo(c, d, "t", 4)
ctx := testNewContext(d)
Expand Down
3 changes: 1 addition & 2 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,8 +154,7 @@ var (
type DDL interface {
CreateSchema(ctx sessionctx.Context, name model.CIStr, charsetInfo *ast.CharsetOpt) error
DropSchema(ctx sessionctx.Context, schema model.CIStr) error
CreateTable(ctx sessionctx.Context, ident ast.Ident, cols []*ast.ColumnDef,
constrs []*ast.Constraint, options []*ast.TableOption) error
CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error
CreateTableWithLike(ctx sessionctx.Context, ident, referIdent ast.Ident) error
DropTable(ctx sessionctx.Context, tableIdent ast.Ident) (err error)
CreateIndex(ctx sessionctx.Context, tableIdent ast.Ident, unique bool, indexName model.CIStr,
Expand Down
Loading

0 comments on commit 217031a

Please sign in to comment.