Skip to content

Commit

Permalink
*: merge master and fix conflict.
Browse files Browse the repository at this point in the history
  • Loading branch information
siddontang committed Nov 16, 2015
1 parent 7cb89d5 commit 42cea84
Show file tree
Hide file tree
Showing 31 changed files with 1,456 additions and 132 deletions.
2 changes: 2 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -6,3 +6,5 @@
coverage.out
.idea/
*.iml
temp_parser_file
y.output
11 changes: 7 additions & 4 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -17,20 +17,23 @@ all: godep parser build test check

godep:
go get github.com/tools/godep
go get github.com/pingcap/go-hbase
go get github.com/pingcap/go-themis

build:
$(GO) build

install:
$(GO) install ./...

TEMP_FILE = temp_parser_file

parser:
go get github.com/qiuyesuifeng/goyacc
go get github.com/qiuyesuifeng/golex
a=`mktemp temp.XXXXXX`; \
goyacc -o /dev/null -xegen $$a parser/parser.y; \
goyacc -o parser/parser.go -xe $$a parser/parser.y 2>&1 | grep "shift/reduce" | awk '{print} END {if (NR > 0) {print "Find conflict in parser.y. Please check y.output for more information."; exit 1;}}'; \
rm -f $$a; \
goyacc -o /dev/null -xegen $(TEMP_FILE) parser/parser.y; \
goyacc -o parser/parser.go -xe $(TEMP_FILE) parser/parser.y 2>&1 | egrep "(shift|reduce)/reduce" | awk '{print} END {if (NR > 0) {print "Find conflict in parser.y. Please check y.output for more information."; system("rm -f $(TEMP_FILE)"); exit 1;}}';
rm -f $(TEMP_FILE); \
rm -f y.output

@if [ $(ARCH) = $(LINUX) ]; \
Expand Down
29 changes: 25 additions & 4 deletions bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,11 @@ const (
CreateGloablVariablesTable = `CREATE TABLE if not exists mysql.GLOBAL_VARIABLES(
VARIABLE_NAME VARCHAR(64) Not Null PRIMARY KEY,
VARIABLE_VALUE VARCHAR(1024) DEFAULT Null);`
// CreateGloablStatusTable is the SQL statement creates global status variable table in system db.
// TODO: MySQL puts GLOBAL_STATUS table in INFORMATION_SCHEMA db.
CreateGloablStatusTable = `CREATE TABLE if not exists mysql.GLOBAL_STATUS(
VARIABLE_NAME VARCHAR(64) Not Null PRIMARY KEY,
VARIABLE_VALUE VARCHAR(1024) DEFAULT Null);`
// CreateTiDBTable is the SQL statement creates a table in system db.
// This table is a key-value struct contains some information used by TiDB.
// Currently we only put bootstrapped in it which indicates if the system is already bootstrapped.
Expand Down Expand Up @@ -134,7 +139,8 @@ func checkBootstrapped(s Session) (bool, error) {
}

func checkBootstrappedVar(s Session) (bool, error) {
sql := fmt.Sprintf(`SELECT VARIABLE_VALUE FROM %s.%s WHERE VARIABLE_NAME="%s"`, mysql.SystemDB, mysql.TiDBTable, bootstrappedVar)
sql := fmt.Sprintf(`SELECT VARIABLE_VALUE FROM %s.%s WHERE VARIABLE_NAME="%s"`,
mysql.SystemDB, mysql.TiDBTable, bootstrappedVar)
rs, err := s.Execute(sql)
if err != nil {
if terror.TableNotExists.Equal(err) {
Expand Down Expand Up @@ -167,6 +173,8 @@ func doDDLWorks(s Session) {
mustExecute(s, CreateColumnPrivTable)
// Create global systemt variable table.
mustExecute(s, CreateGloablVariablesTable)
// Create global status variable table.
mustExecute(s, CreateGloablStatusTable)
// Create TiDB table.
mustExecute(s, CreateTiDBTable)
}
Expand All @@ -176,7 +184,8 @@ func doDDLWorks(s Session) {
func doDMLWorks(s Session) {
mustExecute(s, "BEGIN")
// Insert a default user with empty password.
mustExecute(s, `INSERT INTO mysql.user VALUES ("localhost", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y"),
mustExecute(s, `INSERT INTO mysql.user VALUES
("localhost", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y"),
("127.0.0.1", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y"),
("::1", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y");`)
// Init global system variable table.
Expand All @@ -185,9 +194,21 @@ func doDMLWorks(s Session) {
value := fmt.Sprintf(`("%s", "%s")`, strings.ToLower(k), v.Value)
values = append(values, value)
}
sql := fmt.Sprintf("INSERT INTO %s.%s VALUES %s;", mysql.SystemDB, mysql.GlobalVariablesTable, strings.Join(values, ", "))
sql := fmt.Sprintf("INSERT INTO %s.%s VALUES %s;", mysql.SystemDB, mysql.GlobalVariablesTable,
strings.Join(values, ", "))
mustExecute(s, sql)
// Init global status variable table.
values = make([]string, 0, len(variable.StatusVars))
for k, v := range variable.StatusVars {
value := fmt.Sprintf(`("%s", "%s")`, strings.ToLower(k), v.Value)
values = append(values, value)
}
sql = fmt.Sprintf("INSERT INTO %s.%s VALUES %s;", mysql.SystemDB, mysql.GlobalStatusTable,
strings.Join(values, ", "))
mustExecute(s, sql)
sql = fmt.Sprintf(`INSERT INTO %s.%s VALUES("%s", "%s", "Bootstrap flag. Do not delete.") ON DUPLICATE KEY UPDATE VARIABLE_VALUE="%s"`, mysql.SystemDB, mysql.TiDBTable, bootstrappedVar, bootstrappedVarTrue, bootstrappedVarTrue)
sql = fmt.Sprintf(`INSERT INTO %s.%s VALUES("%s", "%s", "Bootstrap flag. Do not delete.")
ON DUPLICATE KEY UPDATE VARIABLE_VALUE="%s"`,
mysql.SystemDB, mysql.TiDBTable, bootstrappedVar, bootstrappedVarTrue, bootstrappedVarTrue)
mustExecute(s, sql)
mustExecute(s, "COMMIT")
}
Expand Down
118 changes: 118 additions & 0 deletions kv/cache_snapshot.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
// Copyright 2015 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 kv

import "github.com/juju/errors"

var _ Snapshot = (*cacheSnapshot)(nil)

// cacheSnapshot wraps a snapshot and supports cache for read.
type cacheSnapshot struct {
cache MemBuffer
snapshot Snapshot
}

// NewCacheSnapshot creates a new snapshot with cache embedded.
func NewCacheSnapshot(snapshot Snapshot) Snapshot {
return &cacheSnapshot{
cache: p.Get().(MemBuffer),
snapshot: snapshot,
}
}

// Get gets value from snapshot and saves it in cache.
func (c *cacheSnapshot) Get(k Key) ([]byte, error) {
v, err := c.cache.Get(k)
if IsErrNotFound(err) {
v, err = c.snapshot.Get(k)
if err == nil {
err = c.cache.Set([]byte(k), v)
}
return v, errors.Trace(err)
}
if err != nil {
return nil, errors.Trace(err)
}
return v, nil
}

// BatchGet gets a batch of values from snapshot and saves them in cache.
func (c *cacheSnapshot) BatchGet(keys []Key) (map[string][]byte, error) {
m := make(map[string][]byte)
var missedKeys []Key
for _, k := range keys {
v, err := c.cache.Get(k)
if IsErrNotFound(err) {
missedKeys = append(missedKeys, k)
continue
}
if err != nil {
return nil, errors.Trace(err)
}
if len(v) > 0 {
m[string(k)] = v
}
// If len(v) == 0, it means that the key does not exist.
}

values, err := c.snapshot.BatchGet(missedKeys)
if err != nil {
return nil, errors.Trace(err)
}
for _, k := range missedKeys {
ks := string(k)
if v, ok := values[ks]; ok && len(v) > 0 {
err = c.cache.Set(k, v)
m[ks] = v
} else {
err = c.cache.Set(k, nil)
}
if err != nil {
return nil, errors.Trace(err)
}
}
return m, nil
}

// RangeGet gets values from snapshot and saves them in cache.
// The range should be [start, end] as Snapshot.RangeGet() indicated.
func (c *cacheSnapshot) RangeGet(start, end Key, limit int) (map[string][]byte, error) {
values, err := c.snapshot.RangeGet(start, end, limit)
if err != nil {
return nil, errors.Trace(err)
}
for k, v := range values {
err = c.cache.Set([]byte(k), v)
if err != nil {
return nil, errors.Trace(err)
}
}
return values, nil
}

// NewIterator creates an iterator of snapshot.
func (c *cacheSnapshot) NewIterator(param interface{}) Iterator {
return newUnionIter(c.cache.NewIterator(param), c.snapshot.NewIterator(param))
}

// Release reset membuffer and release snapshot.
func (c *cacheSnapshot) Release() {
if c.cache != nil {
c.cache.Release()
c.cache = nil
}
if c.snapshot != nil {
c.snapshot.Release()
}
}
149 changes: 149 additions & 0 deletions kv/cache_snapshot_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
// Copyright 2015 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 kv

import (
"github.com/juju/errors"
. "github.com/pingcap/check"
)

var _ = Suite(&testCacheSnapshotSuite{})

type testCacheSnapshotSuite struct {
store MemBuffer
cache Snapshot
}

func (s *testCacheSnapshotSuite) SetUpTest(c *C) {
s.store = NewMemDbBuffer()
s.cache = NewCacheSnapshot(&mockSnapshot{s.store})
}

func (s *testCacheSnapshotSuite) TearDownTest(c *C) {
s.cache.Release()
}

func (s *testCacheSnapshotSuite) TestGet(c *C) {
s.store.Set([]byte("1"), []byte("1"))
s.store.Set([]byte("2"), []byte("2"))

v, err := s.cache.Get([]byte("1"))
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, []byte("1"))

s.store.Set([]byte("1"), []byte("3"))
v, err = s.cache.Get([]byte("1"))
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, []byte("1"))

s.store.Set([]byte("2"), []byte("4"))
v, err = s.cache.Get([]byte("2"))
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, []byte("4"))
}

func (s *testCacheSnapshotSuite) TestBatchGet(c *C) {
s.store.Set([]byte("1"), []byte("1"))
s.store.Set([]byte("2"), []byte("2"))

m, err := s.cache.BatchGet([]Key{[]byte("1"), []byte("2"), []byte("3")})
c.Assert(err, IsNil)
c.Assert(m["1"], BytesEquals, []byte("1"))
c.Assert(m["2"], BytesEquals, []byte("2"))
_, exist := m["3"]
c.Assert(exist, IsFalse)

// result should be saved in cache
s.store.Set([]byte("1"), []byte("4"))
v, err := s.cache.Get([]byte("1"))
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, []byte("1"))

// nil result should also be saved in cache
s.store.Set([]byte("3"), []byte("3"))
m, err = s.cache.BatchGet([]Key{[]byte("3")})
c.Assert(err, IsNil)
_, exist = m["3"]
c.Assert(exist, IsFalse)
}

func (s *testCacheSnapshotSuite) TestRangeGet(c *C) {
s.store.Set([]byte("1"), []byte("1"))
s.store.Set([]byte("2"), []byte("2"))
s.store.Set([]byte("3"), []byte("3"))

m, err := s.cache.RangeGet([]byte("1"), []byte("2"), 100)
c.Assert(err, IsNil)
c.Assert(m, HasLen, 2)
c.Assert(m["1"], BytesEquals, []byte("1"))
c.Assert(m["2"], BytesEquals, []byte("2"))

// result should be saved in cache
s.store.Set([]byte("1"), []byte("4"))
v, err := s.cache.Get([]byte("1"))
c.Assert(err, IsNil)
c.Assert(v, BytesEquals, []byte("1"))
}

type mockSnapshot struct {
store MemBuffer
}

func (s *mockSnapshot) Get(k Key) ([]byte, error) {
return s.store.Get(k)
}

func (s *mockSnapshot) BatchGet(keys []Key) (map[string][]byte, error) {
m := make(map[string][]byte)
for _, k := range keys {
v, err := s.store.Get(k)
if IsErrNotFound(err) {
continue
}
if err != nil {
return nil, errors.Trace(err)
}
m[string(k)] = v
}
return m, nil
}

func (s *mockSnapshot) RangeGet(start, end Key, limit int) (map[string][]byte, error) {
m := make(map[string][]byte)
it := s.NewIterator([]byte(start))
defer it.Close()
endKey := string(end)
for i := 0; i < limit; i++ {
if !it.Valid() {
break
}
if it.Key() > endKey {
break
}
m[string(it.Key())] = it.Value()
err := it.Next()
if err != nil {
return nil, err
}
}
return m, nil
}

func (s *mockSnapshot) NewIterator(param interface{}) Iterator {
return s.store.NewIterator(param)
}

func (s *mockSnapshot) Release() {
s.store.Release()
}
Loading

0 comments on commit 42cea84

Please sign in to comment.