Merge branch 'master' into replication

This commit is contained in:
Alain Jobart 2015-04-24 15:49:32 -07:00
Родитель a77526dea2 ab9354613a
Коммит a4e5068cfd
61 изменённых файлов: 1371 добавлений и 1882 удалений

Просмотреть файл

@ -1,2 +1,3 @@
Godeps/_workspace/pkg
Godeps/_workspace/bin
_test

3
.gitignore поставляемый
Просмотреть файл

@ -32,3 +32,6 @@ third_party/acolyte
## vitess.io preview site
preview-vitess.io/
# test.go output files
_test/

Просмотреть файл

@ -1,10 +1,13 @@
FROM vitess/bootstrap:mariadb
# Clear out old tree from bootstrap image.
USER root
RUN rm -rf /vt/src/github.com/youtube/vitess
# Re-copy sources from working tree
COPY . /vt/src/github.com/youtube/vitess
# Fix permissions
USER root
RUN chown -R vitess:vitess /vt
USER vitess

Просмотреть файл

@ -4,7 +4,7 @@
MAKEFLAGS = -s
.PHONY: all build test clean unit_test unit_test_cover unit_test_race queryservice_test integration_test bson proto site_test site_integration_test docker_bootstrap docker_test
.PHONY: all build test clean unit_test unit_test_cover unit_test_race queryservice_test integration_test bson proto site_test site_integration_test docker_bootstrap docker_test docker_unit_test
all: build test
@ -191,3 +191,6 @@ docker_bootstrap:
# Example: $ make docker_test flavor=mariadb
docker_test:
docker/test/run.sh $(flavor) 'make test'
docker_unit_test:
docker/test/run.sh $(flavor) 'make unit_test'

Просмотреть файл

@ -139,9 +139,13 @@ if [ -z "$MYSQL_FLAVOR" ]; then
export MYSQL_FLAVOR=MariaDB
fi
case "$MYSQL_FLAVOR" in
"Mysql56")
echo "Mysql 5.6 support is under development and not supported yet."
exit 1
"MySQL56")
myversion=`$VT_MYSQL_ROOT/bin/mysql --version | grep 'Distrib 5\.6'`
if [ "$myversion" == "" ]; then
echo "Couldn't find MySQL 5.6 in $VT_MYSQL_ROOT. Set VT_MYSQL_ROOT to override search location."
exit 1
fi
echo "Found MySQL 5.6 installation in $VT_MYSQL_ROOT."
;;
"MariaDB")

Просмотреть файл

@ -54,7 +54,7 @@ slow-query-log
slow-query-log-file = {{.SlowLogPath}}
socket = {{.SocketFile}}
sort_buffer_size = 2M
table_cache = 2048
table_open_cache = 2048
thread_cache = 200
thread_concurrency = 2
tmpdir = {{.TmpDir}}

Просмотреть файл

@ -54,7 +54,7 @@ slow-query-log
slow-query-log-file = {{.SlowLogPath}}
socket = {{.SocketFile}}
sort_buffer_size = 2M
table_cache = 2048
table_open_cache = 2048
thread_cache = 200
thread_concurrency = 24
tmpdir = {{.TmpDir}}

Двоичные данные
data/bootstrap/mysql-db-dir_5.6.24.tbz Normal file

Двоичный файл не отображается.

Двоичные данные
data/test/binlog_test/vt-0000062347-bin.000001

Двоичный файл не отображается.

Просмотреть файл

@ -81,7 +81,7 @@ if [ -z "$MYSQL_FLAVOR" ]; then
export MYSQL_FLAVOR=MariaDB
fi
case "$MYSQL_FLAVOR" in
"Mysql56")
"MySQL56")
;;
"MariaDB")
export LD_LIBRARY_PATH=$(prepend_path $LD_LIBRARY_PATH $VT_MYSQL_ROOT/lib)

Просмотреть файл

@ -3,7 +3,8 @@ FROM vitess/bootstrap:common
# Install MariaDB 10.0.x
RUN apt-key adv --recv-keys --keyserver keyserver.ubuntu.com 0xcbcb082a1bb943db && \
add-apt-repository 'deb http://sfo1.mirrors.digitalocean.com/mariadb/repo/10.0/debian wheezy main' && \
apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y mariadb-server libmariadbclient-dev && \
apt-get update && \
DEBIAN_FRONTEND=noninteractive apt-get install -y mariadb-server libmariadbclient-dev && \
rm -rf /var/lib/apt/lists/*
# Bootstrap Vitess
@ -11,4 +12,3 @@ WORKDIR /vt/src/github.com/youtube/vitess
USER vitess
ENV MYSQL_FLAVOR MariaDB
RUN ./bootstrap.sh --skip_root_installs

Просмотреть файл

@ -0,0 +1,14 @@
FROM vitess/bootstrap:common
# Install MySQL 5.6
RUN apt-key adv --recv-keys --keyserver pgp.mit.edu 5072E1F5 && \
add-apt-repository 'deb http://repo.mysql.com/apt/debian/ wheezy mysql-5.6' && \
apt-get update && \
DEBIAN_FRONTEND=noninteractive apt-get install -y mysql-server libmysqlclient-dev && \
rm -rf /var/lib/apt/lists/*
# Bootstrap Vitess
WORKDIR /vt/src/github.com/youtube/vitess
USER vitess
ENV MYSQL_FLAVOR MySQL56
RUN ./bootstrap.sh --skip_root_installs

Просмотреть файл

@ -2,6 +2,7 @@
flavor=$1
cmd=$2
args=
if [[ -z "$flavor" ]]; then
echo "Flavor must be specified as first argument."
@ -20,7 +21,7 @@ fi
# To avoid AUFS permission issues, files must allow access by "other"
chmod -R o=g *
args="-ti --rm -e USER=vitess -v /dev/log:/dev/log"
args="$args --rm -e USER=vitess -v /dev/log:/dev/log"
args="$args -v $PWD:/tmp/src"
# Mount in host VTDATAROOT if one exists, since it might be a RAM disk or SSD.
@ -33,15 +34,31 @@ if [[ -n "$VTDATAROOT" ]]; then
echo "Mounting host dir $hostdir as VTDATAROOT"
args="$args -v $hostdir:/vt/vtdataroot --name=$testid -h $testid"
else
args="$args -h test"
testid=test-$$
args="$args --name=$testid -h $testid"
fi
# Run tests
echo "Running tests in vitess/bootstrap:$flavor image..."
docker run $args vitess/bootstrap:$flavor \
bash -c "rm -rf * && cp -R /tmp/src/* . && rm -rf Godeps/_workspace/pkg && $cmd"
bashcmd="rm -rf * && cp -R /tmp/src/* . && rm -rf Godeps/_workspace/pkg && $cmd"
if tty -s; then
# interactive shell
docker run -ti $args vitess/bootstrap:$flavor bash -c "$bashcmd"
exitcode=$?
else
# non-interactive shell (kill child on signal)
trap 'docker rm -f $testid 2>/dev/null' SIGTERM SIGINT
docker run $args vitess/bootstrap:$flavor bash -c "$bashcmd" &
wait $!
exitcode=$?
fi
# Clean up host dir mounted VTDATAROOT
if [[ -n "$hostdir" ]]; then
# Use Docker user to clean up first, to avoid permission errors.
docker run $args vitess/bootstrap:$flavor bash -c 'rm -rf /vt/vtdataroot/*'
rm -rf $hostdir
fi
exit $exitcode

Просмотреть файл

@ -50,6 +50,9 @@ const (
// ErrOptionPreventsStatement is C.ER_OPTION_PREVENTS_STATEMENT
ErrOptionPreventsStatement = C.ER_OPTION_PREVENTS_STATEMENT
// ErrServerLost is C.CR_SERVER_LOST (2013)
ErrServerLost = C.CR_SERVER_LOST
// RedactedPassword is the password value used in redacted configs
RedactedPassword = "****"
)

Просмотреть файл

@ -3,6 +3,7 @@
// license that can be found in the LICENSE file.
#include <mysql.h>
#include <errmsg.h>
// This API provides convenient C wrapper functions for mysql client.

Просмотреть файл

@ -15,7 +15,17 @@
// These low-level vio functions are not declared
// anywhere in the libmysqlclient headers.
#if MYSQL_VERSION_ID >= 100000 // MariaDB 10.0+
// MariaDB has a function vio_socket_shutdown that does what we want.
int vio_socket_shutdown(Vio *vio, int how);
#else
// MySQL 5.6 doesn't have a vio function that just calls shutdown without also
// closing the socket. So we use the system-level shutdown() call, and ask Vio
// to just give us the FD.
my_socket vio_fd(Vio* vio);
#include <sys/socket.h>
#define vio_socket_shutdown(vio, how) shutdown(vio_fd(vio), how)
#endif
// cli_safe_read is declared in sql_common.h.
unsigned long cli_safe_read(MYSQL *mysql);

Просмотреть файл

@ -10,17 +10,17 @@ import (
)
const (
testDuration = 20 * time.Millisecond
testVariance = 5 * time.Millisecond
testDuration = 100 * time.Millisecond
testVariance = 20 * time.Millisecond
)
func TestTick(t *testing.T) {
tkr := NewRandTicker(testDuration, testVariance)
for i := 0; i < 10; i++ {
for i := 0; i < 5; i++ {
start := time.Now()
end := <-tkr.C
diff := start.Add(testDuration).Sub(end)
tolerance := testVariance + 3*time.Millisecond
tolerance := testVariance + 20*time.Millisecond
if diff < -tolerance || diff > tolerance {
t.Errorf("start: %v, end: %v, diff %v. Want <%v tolerenace", start, end, diff, tolerance)
}

Просмотреть файл

@ -1,70 +0,0 @@
package binlog
import (
"encoding/binary"
"os"
"testing"
"github.com/youtube/vitess/go/sync2"
"github.com/youtube/vitess/go/testfiles"
"github.com/youtube/vitess/go/vt/binlog/proto"
"github.com/youtube/vitess/go/vt/mysqlctl"
)
func readEvents(b *testing.B, filename string) <-chan proto.BinlogEvent {
events := make(chan proto.BinlogEvent)
go func() {
defer close(events)
file, err := os.Open(filename)
if err != nil {
b.Fatalf("can't open file %s: %v", filename, err)
}
defer file.Close()
// skip binlog magic header
file.Seek(4, os.SEEK_SET)
for {
// read event header
header := make([]byte, 19)
if _, err := file.Read(header); err != nil {
return
}
// get total event size
size := binary.LittleEndian.Uint32(header[9 : 9+4])
// read the rest of the event
buf := make([]byte, size)
copy(buf[:19], header)
if _, err := file.Read(buf[19:]); err != nil {
return
}
// convert to a BinlogEvent
events <- mysqlctl.NewGoogleBinlogEvent(buf)
}
}()
return events
}
func BenchmarkBinlogStreamerParseEvents(b *testing.B) {
filename := testfiles.Locate("binlog_test/vt-0000062347-bin.000001")
var svm sync2.ServiceManager
count := 0
bls := &BinlogStreamer{dbname: "vt_test_database", sendTransaction: func(tx *proto.BinlogTransaction) error {
count++
return nil
}}
for i := 0; i < b.N; i++ {
events := readEvents(b, filename)
svm.Go(func(svc *sync2.ServiceContext) error {
_, err := bls.parseEvents(svc, events)
return err
})
if err := svm.Join(); err != ServerEOF {
b.Errorf("%v", err)
}
}
b.Logf("%d transactions processed", count)
}

Просмотреть файл

@ -5,9 +5,11 @@
package binlog
import (
"errors"
"fmt"
"io"
"reflect"
"strings"
"testing"
"time"
@ -18,50 +20,137 @@ import (
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
)
// sample Google MySQL event data
var (
rotateEvent = []byte{0x0, 0x0, 0x0, 0x0, 0x4, 0x88, 0xf3, 0x0, 0x0, 0x33, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x20, 0x0, 0x23, 0x3, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x76, 0x74, 0x2d, 0x30, 0x30, 0x30, 0x30, 0x30, 0x36, 0x32, 0x33, 0x34, 0x34, 0x2d, 0x62, 0x69, 0x6e, 0x2e, 0x30, 0x30, 0x30, 0x30, 0x30, 0x31}
formatEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0xf, 0x88, 0xf3, 0x0, 0x0, 0x66, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4, 0x0, 0x35, 0x2e, 0x31, 0x2e, 0x36, 0x33, 0x2d, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2d, 0x6c, 0x6f, 0x67, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1b, 0x38, 0xd, 0x0, 0x8, 0x0, 0x12, 0x0, 0x4, 0x4, 0x4, 0x4, 0x12, 0x0, 0x0, 0x53, 0x0, 0x4, 0x1a, 0x8, 0x0, 0x0, 0x0, 0x8, 0x8, 0x8, 0x2}
beginEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0x58, 0x0, 0x0, 0x0, 0xc2, 0x0, 0x0, 0x0, 0x8, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x42, 0x45, 0x47, 0x49, 0x4e}
commitEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0x59, 0x0, 0x0, 0x0, 0xc2, 0x0, 0x0, 0x0, 0x8, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54}
rollbackEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0x5b, 0x0, 0x0, 0x0, 0xc2, 0x0, 0x0, 0x0, 0x8, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x52, 0x4f, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b}
insertEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0x9f, 0x0, 0x0, 0x0, 0x61, 0x1, 0x0, 0x0, 0x0, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x20, 0x69, 0x6e, 0x74, 0x6f, 0x20, 0x76, 0x74, 0x5f, 0x61, 0x28, 0x65, 0x69, 0x64, 0x2c, 0x20, 0x69, 0x64, 0x29, 0x20, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x20, 0x28, 0x31, 0x2c, 0x20, 0x31, 0x29, 0x20, 0x2f, 0x2a, 0x20, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x20, 0x76, 0x74, 0x5f, 0x61, 0x20, 0x28, 0x65, 0x69, 0x64, 0x20, 0x69, 0x64, 0x20, 0x29, 0x20, 0x28, 0x31, 0x20, 0x31, 0x20, 0x29, 0x3b, 0x20, 0x2a, 0x2f}
createEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0xca, 0x0, 0x0, 0x0, 0xed, 0x3, 0x0, 0x0, 0x0, 0x0, 0xa, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x8, 0x0, 0x8, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x20, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x20, 0x69, 0x66, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x20, 0x28, 0xa, 0x69, 0x64, 0x20, 0x62, 0x69, 0x67, 0x69, 0x6e, 0x74, 0x20, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2c, 0xa, 0x6d, 0x73, 0x67, 0x20, 0x76, 0x61, 0x72, 0x63, 0x68, 0x61, 0x72, 0x28, 0x36, 0x34, 0x29, 0x2c, 0xa, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x20, 0x6b, 0x65, 0x79, 0x20, 0x28, 0x69, 0x64, 0x29, 0xa, 0x29, 0x20, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x3d, 0x49, 0x6e, 0x6e, 0x6f, 0x44, 0x42}
xidEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0x10, 0x88, 0xf3, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x4e, 0xa, 0x0, 0x0, 0x0, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x78, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}
insertIDEvent = []byte{0xea, 0xa8, 0xea, 0x53, 0x5, 0x88, 0xf3, 0x0, 0x0, 0x24, 0x0, 0x0, 0x0, 0xb8, 0x6, 0x0, 0x0, 0x0, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x2, 0x65, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}
otherDBEvent = []byte{0x86, 0x11, 0xed, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0xc8, 0x0, 0x0, 0x0, 0x30, 0x6, 0x0, 0x0, 0x0, 0x0, 0xf, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xe, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x8, 0x0, 0x8, 0x0, 0x21, 0x0, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x0, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x20, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x20, 0x69, 0x66, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x20, 0x28, 0xa, 0x69, 0x64, 0x20, 0x62, 0x69, 0x67, 0x69, 0x6e, 0x74, 0x20, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2c, 0xa, 0x6d, 0x73, 0x67, 0x20, 0x76, 0x61, 0x72, 0x63, 0x68, 0x61, 0x72, 0x28, 0x36, 0x34, 0x29, 0x2c, 0xa, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x20, 0x6b, 0x65, 0x79, 0x20, 0x28, 0x69, 0x64, 0x29, 0xa, 0x29, 0x20, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x3d, 0x49, 0x6e, 0x6e, 0x6f, 0x44, 0x42}
otherDBBeginEvent = []byte{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0x56, 0x0, 0x0, 0x0, 0xc2, 0x0, 0x0, 0x0, 0x8, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xe, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x42, 0x45, 0x47, 0x49, 0x4e}
// fakeEvent implements proto.BinlogEvent.
type fakeEvent struct{}
mariadbRotateEvent = []byte{0x0, 0x0, 0x0, 0x0, 0x4, 0x88, 0xf3, 0x0, 0x0, 0x33, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x20, 0x0, 0x4, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x76, 0x74, 0x2d, 0x30, 0x30, 0x30, 0x30, 0x30, 0x36, 0x32, 0x33, 0x34, 0x34, 0x2d, 0x62, 0x69, 0x6e, 0x2e, 0x30, 0x30, 0x30, 0x30, 0x30, 0x31}
mariadbFormatEvent = []byte{0x87, 0x41, 0x9, 0x54, 0xf, 0x88, 0xf3, 0x0, 0x0, 0xf4, 0x0, 0x0, 0x0, 0xf8, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4, 0x0, 0x31, 0x30, 0x2e, 0x30, 0x2e, 0x31, 0x33, 0x2d, 0x4d, 0x61, 0x72, 0x69, 0x61, 0x44, 0x42, 0x2d, 0x31, 0x7e, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x65, 0x2d, 0x6c, 0x6f, 0x67, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x87, 0x41, 0x9, 0x54, 0x13, 0x38, 0xd, 0x0, 0x8, 0x0, 0x12, 0x0, 0x4, 0x4, 0x4, 0x4, 0x12, 0x0, 0x0, 0xdc, 0x0, 0x4, 0x1a, 0x8, 0x0, 0x0, 0x0, 0x8, 0x8, 0x8, 0x2, 0x0, 0x0, 0x0, 0xa, 0xa, 0xa, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4, 0x13, 0x4, 0x0, 0x6e, 0xe0, 0xfd, 0x41}
mariadbStandaloneGTIDEvent = []byte{0x88, 0x41, 0x9, 0x54, 0xa2, 0x88, 0xf3, 0x0, 0x0, 0x26, 0x0, 0x0, 0x0, 0xcf, 0x8, 0x0, 0x0, 0x8, 0x0, 0x9, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}
mariadbBeginGTIDEvent = []byte{0x88, 0x41, 0x9, 0x54, 0xa2, 0x88, 0xf3, 0x0, 0x0, 0x26, 0x0, 0x0, 0x0, 0xb5, 0x9, 0x0, 0x0, 0x8, 0x0, 0xa, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}
mariadbCreateEvent = []byte{0x88, 0x41, 0x9, 0x54, 0x2, 0x88, 0xf3, 0x0, 0x0, 0xc2, 0x0, 0x0, 0x0, 0xf2, 0x6, 0x0, 0x0, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x8, 0x0, 0x8, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x20, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x20, 0x69, 0x66, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x20, 0x28, 0xa, 0x69, 0x64, 0x20, 0x62, 0x69, 0x67, 0x69, 0x6e, 0x74, 0x20, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2c, 0xa, 0x6d, 0x73, 0x67, 0x20, 0x76, 0x61, 0x72, 0x63, 0x68, 0x61, 0x72, 0x28, 0x36, 0x34, 0x29, 0x2c, 0xa, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x20, 0x6b, 0x65, 0x79, 0x20, 0x28, 0x69, 0x64, 0x29, 0xa, 0x29, 0x20, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x3d, 0x49, 0x6e, 0x6e, 0x6f, 0x44, 0x42}
mariadbInsertEvent = []byte{0x88, 0x41, 0x9, 0x54, 0x2, 0x88, 0xf3, 0x0, 0x0, 0xa8, 0x0, 0x0, 0x0, 0x79, 0xa, 0x0, 0x0, 0x0, 0x0, 0x27, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x20, 0x69, 0x6e, 0x74, 0x6f, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x28, 0x6d, 0x73, 0x67, 0x29, 0x20, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x20, 0x28, 0x27, 0x74, 0x65, 0x73, 0x74, 0x20, 0x30, 0x27, 0x29, 0x20, 0x2f, 0x2a, 0x20, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x20, 0x28, 0x69, 0x64, 0x20, 0x29, 0x20, 0x28, 0x6e, 0x75, 0x6c, 0x6c, 0x20, 0x29, 0x3b, 0x20, 0x2a, 0x2f}
mariadbXidEvent = []byte{0x88, 0x41, 0x9, 0x54, 0x10, 0x88, 0xf3, 0x0, 0x0, 0x1b, 0x0, 0x0, 0x0, 0xe0, 0xc, 0x0, 0x0, 0x0, 0x0, 0x85, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}
)
func (fakeEvent) IsValid() bool { return true }
func (fakeEvent) IsFormatDescription() bool { return false }
func (fakeEvent) IsQuery() bool { return false }
func (fakeEvent) IsXID() bool { return false }
func (fakeEvent) IsGTID() bool { return false }
func (fakeEvent) IsRotate() bool { return false }
func (fakeEvent) IsIntVar() bool { return false }
func (fakeEvent) IsRand() bool { return false }
func (fakeEvent) HasGTID(proto.BinlogFormat) bool { return true }
func (fakeEvent) Timestamp() uint32 { return 1407805592 }
func (fakeEvent) Format() (proto.BinlogFormat, error) {
return proto.BinlogFormat{}, errors.New("not a format")
}
func (fakeEvent) GTID(proto.BinlogFormat) (myproto.GTID, error) {
return myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0xd}, nil
}
func (fakeEvent) IsBeginGTID(proto.BinlogFormat) bool { return false }
func (fakeEvent) Query(proto.BinlogFormat) (proto.Query, error) {
return proto.Query{}, errors.New("not a query")
}
func (fakeEvent) IntVar(proto.BinlogFormat) (string, uint64, error) {
return "", 0, errors.New("not an intvar")
}
func (fakeEvent) Rand(proto.BinlogFormat) (uint64, uint64, error) {
return 0, 0, errors.New("not a rand")
}
func (ev fakeEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) { return ev, nil }
func sendTestEvents(channel chan<- proto.BinlogEvent, events [][]byte) {
for _, buf := range events {
channel <- mysqlctl.NewGoogleBinlogEvent(buf)
}
close(channel)
type invalidEvent struct{ fakeEvent }
func (invalidEvent) IsValid() bool { return false }
func (ev invalidEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) { return ev, nil }
type rotateEvent struct{ fakeEvent }
func (rotateEvent) IsRotate() bool { return true }
func (ev rotateEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) { return ev, nil }
type formatEvent struct{ fakeEvent }
func (formatEvent) IsFormatDescription() bool { return true }
func (formatEvent) Format() (proto.BinlogFormat, error) {
return proto.BinlogFormat{FormatVersion: 1}, nil
}
func (ev formatEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) { return ev, nil }
type invalidFormatEvent struct{ formatEvent }
func (invalidFormatEvent) Format() (proto.BinlogFormat, error) {
return proto.BinlogFormat{}, errors.New("invalid format event")
}
func (ev invalidFormatEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) {
return ev, nil
}
func sendMariadbTestEvents(channel chan<- proto.BinlogEvent, events [][]byte) {
for _, buf := range events {
channel <- mysqlctl.NewMariadbBinlogEvent(buf)
type queryEvent struct {
fakeEvent
query proto.Query
}
func (queryEvent) IsQuery() bool { return true }
func (ev queryEvent) Query(proto.BinlogFormat) (proto.Query, error) {
return ev.query, nil
}
func (ev queryEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) { return ev, nil }
type invalidQueryEvent struct{ queryEvent }
func (invalidQueryEvent) Query(proto.BinlogFormat) (proto.Query, error) {
return proto.Query{}, errors.New("invalid query event")
}
func (ev invalidQueryEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) {
return ev, nil
}
type xidEvent struct{ fakeEvent }
func (xidEvent) IsXID() bool { return true }
func (ev xidEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) { return ev, nil }
type intVarEvent struct {
fakeEvent
name string
value uint64
}
func (intVarEvent) IsIntVar() bool { return true }
func (ev intVarEvent) IntVar(proto.BinlogFormat) (string, uint64, error) {
return ev.name, ev.value, nil
}
func (ev intVarEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) { return ev, nil }
type invalidIntVarEvent struct{ intVarEvent }
func (invalidIntVarEvent) IntVar(proto.BinlogFormat) (string, uint64, error) {
return "", 0, errors.New("invalid intvar event")
}
func (ev invalidIntVarEvent) StripChecksum(proto.BinlogFormat) (proto.BinlogEvent, []byte) {
return ev, nil
}
// sample MariaDB event data
var (
mariadbRotateEvent = mysqlctl.NewMariadbBinlogEvent([]byte{0x0, 0x0, 0x0, 0x0, 0x4, 0x88, 0xf3, 0x0, 0x0, 0x33, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x20, 0x0, 0x4, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x76, 0x74, 0x2d, 0x30, 0x30, 0x30, 0x30, 0x30, 0x36, 0x32, 0x33, 0x34, 0x34, 0x2d, 0x62, 0x69, 0x6e, 0x2e, 0x30, 0x30, 0x30, 0x30, 0x30, 0x31})
mariadbFormatEvent = mysqlctl.NewMariadbBinlogEvent([]byte{0x87, 0x41, 0x9, 0x54, 0xf, 0x88, 0xf3, 0x0, 0x0, 0xf4, 0x0, 0x0, 0x0, 0xf8, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4, 0x0, 0x31, 0x30, 0x2e, 0x30, 0x2e, 0x31, 0x33, 0x2d, 0x4d, 0x61, 0x72, 0x69, 0x61, 0x44, 0x42, 0x2d, 0x31, 0x7e, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x65, 0x2d, 0x6c, 0x6f, 0x67, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x87, 0x41, 0x9, 0x54, 0x13, 0x38, 0xd, 0x0, 0x8, 0x0, 0x12, 0x0, 0x4, 0x4, 0x4, 0x4, 0x12, 0x0, 0x0, 0xdc, 0x0, 0x4, 0x1a, 0x8, 0x0, 0x0, 0x0, 0x8, 0x8, 0x8, 0x2, 0x0, 0x0, 0x0, 0xa, 0xa, 0xa, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4, 0x13, 0x4, 0x0, 0x6e, 0xe0, 0xfd, 0x41})
mariadbStandaloneGTIDEvent = mysqlctl.NewMariadbBinlogEvent([]byte{0x88, 0x41, 0x9, 0x54, 0xa2, 0x88, 0xf3, 0x0, 0x0, 0x26, 0x0, 0x0, 0x0, 0xcf, 0x8, 0x0, 0x0, 0x8, 0x0, 0x9, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0})
mariadbBeginGTIDEvent = mysqlctl.NewMariadbBinlogEvent([]byte{0x88, 0x41, 0x9, 0x54, 0xa2, 0x88, 0xf3, 0x0, 0x0, 0x26, 0x0, 0x0, 0x0, 0xb5, 0x9, 0x0, 0x0, 0x8, 0x0, 0xa, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0})
mariadbCreateEvent = mysqlctl.NewMariadbBinlogEvent([]byte{0x88, 0x41, 0x9, 0x54, 0x2, 0x88, 0xf3, 0x0, 0x0, 0xc2, 0x0, 0x0, 0x0, 0xf2, 0x6, 0x0, 0x0, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x8, 0x0, 0x8, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x20, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x20, 0x69, 0x66, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x20, 0x28, 0xa, 0x69, 0x64, 0x20, 0x62, 0x69, 0x67, 0x69, 0x6e, 0x74, 0x20, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2c, 0xa, 0x6d, 0x73, 0x67, 0x20, 0x76, 0x61, 0x72, 0x63, 0x68, 0x61, 0x72, 0x28, 0x36, 0x34, 0x29, 0x2c, 0xa, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x20, 0x6b, 0x65, 0x79, 0x20, 0x28, 0x69, 0x64, 0x29, 0xa, 0x29, 0x20, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x3d, 0x49, 0x6e, 0x6e, 0x6f, 0x44, 0x42})
mariadbInsertEvent = mysqlctl.NewMariadbBinlogEvent([]byte{0x88, 0x41, 0x9, 0x54, 0x2, 0x88, 0xf3, 0x0, 0x0, 0xa8, 0x0, 0x0, 0x0, 0x79, 0xa, 0x0, 0x0, 0x0, 0x0, 0x27, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x20, 0x69, 0x6e, 0x74, 0x6f, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x28, 0x6d, 0x73, 0x67, 0x29, 0x20, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x20, 0x28, 0x27, 0x74, 0x65, 0x73, 0x74, 0x20, 0x30, 0x27, 0x29, 0x20, 0x2f, 0x2a, 0x20, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x20, 0x76, 0x74, 0x5f, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x20, 0x28, 0x69, 0x64, 0x20, 0x29, 0x20, 0x28, 0x6e, 0x75, 0x6c, 0x6c, 0x20, 0x29, 0x3b, 0x20, 0x2a, 0x2f})
mariadbXidEvent = mysqlctl.NewMariadbBinlogEvent([]byte{0x88, 0x41, 0x9, 0x54, 0x10, 0x88, 0xf3, 0x0, 0x0, 0x1b, 0x0, 0x0, 0x0, 0xe0, 0xc, 0x0, 0x0, 0x0, 0x0, 0x85, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0})
charset = &mproto.Charset{Client: 33, Conn: 33, Server: 33}
)
func sendTestEvents(channel chan<- proto.BinlogEvent, events []proto.BinlogEvent) {
for _, ev := range events {
channel <- ev
}
close(channel)
}
func TestBinlogStreamerParseEventsXID(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
events := make(chan proto.BinlogEvent)
@ -69,12 +158,12 @@ func TestBinlogStreamerParseEventsXID(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -100,12 +189,12 @@ func TestBinlogStreamerParseEventsXID(t *testing.T) {
}
func TestBinlogStreamerParseEventsCommit(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertEvent,
commitEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("COMMIT")}},
}
events := make(chan proto.BinlogEvent)
@ -113,12 +202,12 @@ func TestBinlogStreamerParseEventsCommit(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -175,12 +264,12 @@ func TestBinlogStreamerStop(t *testing.T) {
}
func TestBinlogStreamerParseEventsClientEOF(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
want := ClientEOF
@ -232,12 +321,12 @@ func TestBinlogStreamerParseEventsServerEOF(t *testing.T) {
}
func TestBinlogStreamerParseEventsSendErrorXID(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
want := "send reply error: foobar"
@ -265,12 +354,12 @@ func TestBinlogStreamerParseEventsSendErrorXID(t *testing.T) {
}
func TestBinlogStreamerParseEventsSendErrorCommit(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertEvent,
commitEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("COMMIT")}},
}
want := "send reply error: foobar"
@ -298,16 +387,14 @@ func TestBinlogStreamerParseEventsSendErrorCommit(t *testing.T) {
}
func TestBinlogStreamerParseEventsInvalid(t *testing.T) {
invalidEvent := rotateEvent[:19]
input := [][]byte{
invalidEvent,
formatEvent,
beginEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
invalidEvent{},
xidEvent{},
}
want := "can't parse binlog event, invalid data: mysqlctl.googleBinlogEvent{binlogEvent:mysqlctl.binlogEvent{0x0, 0x0, 0x0, 0x0, 0x4, 0x88, 0xf3, 0x0, 0x0, 0x33, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x20, 0x0}}"
want := "can't parse binlog event, invalid data:"
events := make(chan proto.BinlogEvent)
@ -327,24 +414,20 @@ func TestBinlogStreamerParseEventsInvalid(t *testing.T) {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
if got := err.Error(); !strings.HasPrefix(got, want) {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestBinlogStreamerParseEventsInvalidFormat(t *testing.T) {
invalidEvent := make([]byte, len(formatEvent))
copy(invalidEvent, formatEvent)
invalidEvent[19+2+50+4] = 12 // mess up the HeaderLength
input := [][]byte{
rotateEvent,
invalidEvent,
beginEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
invalidFormatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
want := "can't parse FORMAT_DESCRIPTION_EVENT: header length = 12, should be >= 19, event data: mysqlctl.googleBinlogEvent{binlogEvent:mysqlctl.binlogEvent{0x98, 0x68, 0xe9, 0x53, 0xf, 0x88, 0xf3, 0x0, 0x0, 0x66, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x4, 0x0, 0x35, 0x2e, 0x31, 0x2e, 0x36, 0x33, 0x2d, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2d, 0x6c, 0x6f, 0x67, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc, 0x38, 0xd, 0x0, 0x8, 0x0, 0x12, 0x0, 0x4, 0x4, 0x4, 0x4, 0x12, 0x0, 0x0, 0x53, 0x0, 0x4, 0x1a, 0x8, 0x0, 0x0, 0x0, 0x8, 0x8, 0x8, 0x2}}"
want := "can't parse FORMAT_DESCRIPTION_EVENT:"
events := make(chan proto.BinlogEvent)
@ -364,20 +447,20 @@ func TestBinlogStreamerParseEventsInvalidFormat(t *testing.T) {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
if got := err.Error(); !strings.HasPrefix(got, want) {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestBinlogStreamerParseEventsNoFormat(t *testing.T) {
input := [][]byte{
rotateEvent,
//formatEvent,
beginEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
//formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
want := "got a real event before FORMAT_DESCRIPTION_EVENT: mysqlctl.googleBinlogEvent{binlogEvent:mysqlctl.binlogEvent{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0x58, 0x0, 0x0, 0x0, 0xc2, 0x0, 0x0, 0x0, 0x8, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x10, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x42, 0x45, 0x47, 0x49, 0x4e}}"
want := "got a real event before FORMAT_DESCRIPTION_EVENT:"
events := make(chan proto.BinlogEvent)
@ -397,24 +480,20 @@ func TestBinlogStreamerParseEventsNoFormat(t *testing.T) {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
if got := err.Error(); !strings.HasPrefix(got, want) {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestBinlogStreamerParseEventsInvalidQuery(t *testing.T) {
invalidEvent := make([]byte, len(insertEvent))
copy(invalidEvent, insertEvent)
invalidEvent[19+8+4+4] = 200 // mess up the db_name length
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
invalidEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
invalidQueryEvent{},
xidEvent{},
}
want := "can't get query from binlog event: SQL query position overflows buffer (240 > 132), event data: mysqlctl.googleBinlogEvent{binlogEvent:mysqlctl.binlogEvent{0x98, 0x68, 0xe9, 0x53, 0x2, 0x88, 0xf3, 0x0, 0x0, 0x9f, 0x0, 0x0, 0x0, 0x61, 0x1, 0x0, 0x0, 0x0, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x23, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc8, 0x0, 0x0, 0x1a, 0x0, 0x0, 0x0, 0x40, 0x0, 0x0, 0x1, 0x0, 0x0, 0x20, 0x0, 0x0, 0x0, 0x0, 0x0, 0x6, 0x3, 0x73, 0x74, 0x64, 0x4, 0x21, 0x0, 0x21, 0x0, 0x21, 0x0, 0x76, 0x74, 0x5f, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x0, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x20, 0x69, 0x6e, 0x74, 0x6f, 0x20, 0x76, 0x74, 0x5f, 0x61, 0x28, 0x65, 0x69, 0x64, 0x2c, 0x20, 0x69, 0x64, 0x29, 0x20, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x20, 0x28, 0x31, 0x2c, 0x20, 0x31, 0x29, 0x20, 0x2f, 0x2a, 0x20, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x20, 0x76, 0x74, 0x5f, 0x61, 0x20, 0x28, 0x65, 0x69, 0x64, 0x20, 0x69, 0x64, 0x20, 0x29, 0x20, 0x28, 0x31, 0x20, 0x31, 0x20, 0x29, 0x3b, 0x20, 0x2a, 0x2f}}"
want := "can't get query from binlog event:"
events := make(chan proto.BinlogEvent)
@ -434,22 +513,22 @@ func TestBinlogStreamerParseEventsInvalidQuery(t *testing.T) {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
if got := err.Error(); !strings.HasPrefix(got, want) {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestBinlogStreamerParseEventsRollback(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertEvent,
insertEvent,
rollbackEvent,
beginEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("ROLLBACK")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
events := make(chan proto.BinlogEvent)
@ -459,16 +538,16 @@ func TestBinlogStreamerParseEventsRollback(t *testing.T) {
Statements: nil,
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -494,11 +573,11 @@ func TestBinlogStreamerParseEventsRollback(t *testing.T) {
}
func TestBinlogStreamerParseEventsDMLWithoutBegin(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
events := make(chan proto.BinlogEvent)
@ -506,18 +585,18 @@ func TestBinlogStreamerParseEventsDMLWithoutBegin(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
proto.BinlogTransaction{
Statements: nil,
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -543,12 +622,12 @@ func TestBinlogStreamerParseEventsDMLWithoutBegin(t *testing.T) {
}
func TestBinlogStreamerParseEventsBeginWithoutCommit(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
insertEvent,
beginEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
xidEvent{},
}
events := make(chan proto.BinlogEvent)
@ -556,18 +635,18 @@ func TestBinlogStreamerParseEventsBeginWithoutCommit(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
proto.BinlogTransaction{
Statements: []proto.Statement{},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -593,13 +672,13 @@ func TestBinlogStreamerParseEventsBeginWithoutCommit(t *testing.T) {
}
func TestBinlogStreamerParseEventsSetInsertID(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertIDEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
intVarEvent{name: "INSERT_ID", value: 101},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
events := make(chan proto.BinlogEvent)
@ -607,13 +686,13 @@ func TestBinlogStreamerParseEventsSetInsertID(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: nil, Sql: []byte("SET INSERT_ID=101")},
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET INSERT_ID=101")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -639,19 +718,15 @@ func TestBinlogStreamerParseEventsSetInsertID(t *testing.T) {
}
func TestBinlogStreamerParseEventsInvalidIntVar(t *testing.T) {
invalidEvent := make([]byte, len(insertIDEvent))
copy(invalidEvent, insertIDEvent)
invalidEvent[19+8] = 3 // mess up the variable ID
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
invalidEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
invalidIntVarEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
want := "can't parse INTVAR_EVENT: invalid IntVar ID: 3, event data: mysqlctl.googleBinlogEvent{binlogEvent:mysqlctl.binlogEvent{0xea, 0xa8, 0xea, 0x53, 0x5, 0x88, 0xf3, 0x0, 0x0, 0x24, 0x0, 0x0, 0x0, 0xb8, 0x6, 0x0, 0x0, 0x0, 0x0, 0xd, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x3, 0x65, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}}"
want := "can't parse INTVAR_EVENT:"
events := make(chan proto.BinlogEvent)
@ -671,19 +746,19 @@ func TestBinlogStreamerParseEventsInvalidIntVar(t *testing.T) {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
if got := err.Error(); !strings.HasPrefix(got, want) {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestBinlogStreamerParseEventsOtherDB(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
otherDBEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "other", Sql: []byte("INSERT INTO test values (3, 4)")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
events := make(chan proto.BinlogEvent)
@ -691,12 +766,12 @@ func TestBinlogStreamerParseEventsOtherDB(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -722,13 +797,13 @@ func TestBinlogStreamerParseEventsOtherDB(t *testing.T) {
}
func TestBinlogStreamerParseEventsOtherDBBegin(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
otherDBBeginEvent, // Check that this doesn't get filtered out.
otherDBEvent,
insertEvent,
xidEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "other", Sql: []byte("BEGIN")}}, // Check that this doesn't get filtered out.
queryEvent{query: proto.Query{Database: "other", Sql: []byte("INSERT INTO test values (3, 4)")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
xidEvent{},
}
events := make(chan proto.BinlogEvent)
@ -736,12 +811,12 @@ func TestBinlogStreamerParseEventsOtherDBBegin(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
proto.Statement{Category: proto.BL_SET, Sql: []byte("SET TIMESTAMP=1407805592")},
proto.Statement{Category: proto.BL_DML, Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")},
},
Timestamp: 1407805592,
GTIDField: myproto.GTIDField{
Value: myproto.GoogleGTID{ServerID: 62344, GroupID: 0x0d}},
Value: myproto.MariadbGTID{Domain: 0, Server: 62344, Sequence: 0x0d}},
},
}
var got []proto.BinlogTransaction
@ -767,12 +842,12 @@ func TestBinlogStreamerParseEventsOtherDBBegin(t *testing.T) {
}
func TestBinlogStreamerParseEventsBeginAgain(t *testing.T) {
input := [][]byte{
rotateEvent,
formatEvent,
beginEvent,
insertEvent,
beginEvent,
input := []proto.BinlogEvent{
rotateEvent{},
formatEvent{},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("insert into vt_a(eid, id) values (1, 1) /* _stream vt_a (eid id ) (1 1 ); */")}},
queryEvent{query: proto.Query{Database: "vt_test_keyspace", Sql: []byte("BEGIN")}},
}
events := make(chan proto.BinlogEvent)
@ -799,7 +874,7 @@ func TestBinlogStreamerParseEventsBeginAgain(t *testing.T) {
}
func TestBinlogStreamerParseEventsMariadbBeginGTID(t *testing.T) {
input := [][]byte{
input := []proto.BinlogEvent{
mariadbRotateEvent,
mariadbFormatEvent,
mariadbBeginGTIDEvent,
@ -812,8 +887,8 @@ func TestBinlogStreamerParseEventsMariadbBeginGTID(t *testing.T) {
want := []proto.BinlogTransaction{
proto.BinlogTransaction{
Statements: []proto.Statement{
proto.Statement{Category: proto.BL_SET, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("SET TIMESTAMP=1409892744")},
proto.Statement{Category: proto.BL_DML, Charset: &mproto.Charset{Client: 33, Conn: 33, Server: 33}, Sql: []byte("insert into vt_insert_test(msg) values ('test 0') /* _stream vt_insert_test (id ) (null ); */")},
proto.Statement{Category: proto.BL_SET, Charset: charset, Sql: []byte("SET TIMESTAMP=1409892744")},
proto.Statement{Category: proto.BL_DML, Charset: charset, Sql: []byte("insert into vt_insert_test(msg) values ('test 0') /* _stream vt_insert_test (id ) (null ); */")},
},
Timestamp: 1409892744,
GTIDField: myproto.GTIDField{
@ -827,7 +902,7 @@ func TestBinlogStreamerParseEventsMariadbBeginGTID(t *testing.T) {
}
bls := NewBinlogStreamer("vt_test_keyspace", nil, nil, myproto.ReplicationPosition{}, sendTransaction)
go sendMariadbTestEvents(events, input)
go sendTestEvents(events, input)
svm := &sync2.ServiceManager{}
svm.Go(func(ctx *sync2.ServiceContext) error {
_, err := bls.parseEvents(ctx, events)
@ -843,7 +918,7 @@ func TestBinlogStreamerParseEventsMariadbBeginGTID(t *testing.T) {
}
func TestBinlogStreamerParseEventsMariadbStandaloneGTID(t *testing.T) {
input := [][]byte{
input := []proto.BinlogEvent{
mariadbRotateEvent,
mariadbFormatEvent,
mariadbStandaloneGTIDEvent,
@ -870,7 +945,7 @@ func TestBinlogStreamerParseEventsMariadbStandaloneGTID(t *testing.T) {
}
bls := NewBinlogStreamer("vt_test_keyspace", nil, nil, myproto.ReplicationPosition{}, sendTransaction)
go sendMariadbTestEvents(events, input)
go sendTestEvents(events, input)
svm := &sync2.ServiceManager{}
svm.Go(func(ctx *sync2.ServiceContext) error {
_, err := bls.parseEvents(ctx, events)

Просмотреть файл

@ -11,10 +11,10 @@ import (
)
func TestPopulateBlpCheckpoint(t *testing.T) {
gtid := myproto.MustParseGTID("GoogleMysql", "41983-19283")
gtid := myproto.MustParseGTID("MariaDB", "0-1-1083")
want := "INSERT INTO _vt.blp_checkpoint " +
"(source_shard_uid, pos, time_updated, transaction_timestamp, flags) " +
"VALUES (18372, 'GoogleMysql/41983-19283', 481823, 0, 'myflags')"
"VALUES (18372, 'MariaDB/0-1-1083', 481823, 0, 'myflags')"
got := PopulateBlpCheckpoint(18372, myproto.ReplicationPosition{GTIDSet: gtid.GTIDSet()}, 481823, "myflags")
if got != want {
@ -23,9 +23,9 @@ func TestPopulateBlpCheckpoint(t *testing.T) {
}
func TestUpdateBlpCheckpoint(t *testing.T) {
gtid := myproto.MustParseGTID("GoogleMysql", "41983-58283")
gtid := myproto.MustParseGTID("MariaDB", "0-1-8283")
want := "UPDATE _vt.blp_checkpoint " +
"SET pos='GoogleMysql/41983-58283', time_updated=88822 " +
"SET pos='MariaDB/0-1-8283', time_updated=88822 " +
"WHERE source_shard_uid=78522"
got := UpdateBlpCheckpoint(78522, myproto.ReplicationPosition{GTIDSet: gtid.GTIDSet()}, 88822, 0)
@ -35,9 +35,9 @@ func TestUpdateBlpCheckpoint(t *testing.T) {
}
func TestUpdateBlpCheckpointTimestamp(t *testing.T) {
gtid := myproto.MustParseGTID("GoogleMysql", "41983-58283")
gtid := myproto.MustParseGTID("MariaDB", "0-2-582")
want := "UPDATE _vt.blp_checkpoint " +
"SET pos='GoogleMysql/41983-58283', time_updated=88822, transaction_timestamp=481828 " +
"SET pos='MariaDB/0-2-582', time_updated=88822, transaction_timestamp=481828 " +
"WHERE source_shard_uid=78522"
got := UpdateBlpCheckpoint(78522, myproto.ReplicationPosition{GTIDSet: gtid.GTIDSet()}, 88822, 481828)

Просмотреть файл

@ -102,7 +102,7 @@ func TestDMLEvent(t *testing.T) {
},
},
Timestamp: 1,
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("GoogleMysql", "41983-20")},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("MariaDB", "0-41983-20")},
}
evs := &EventStreamer{
sendEvent: func(event *proto.StreamEvent) error {
@ -120,7 +120,7 @@ func TestDMLEvent(t *testing.T) {
t.Errorf("got %s, want %s", got, want)
}
case "POS":
want := `&{POS [] [] 1 41983-20}`
want := `&{POS [] [] 1 0-41983-20}`
got := fmt.Sprintf("%v", event)
if got != want {
t.Errorf("got %s, want %s", got, want)
@ -149,7 +149,7 @@ func TestDDLEvent(t *testing.T) {
},
},
Timestamp: 1,
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("GoogleMysql", "41983-20")},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("MariaDB", "0-41983-20")},
}
evs := &EventStreamer{
sendEvent: func(event *proto.StreamEvent) error {
@ -161,7 +161,7 @@ func TestDDLEvent(t *testing.T) {
t.Errorf("got %s, want %s", got, want)
}
case "POS":
want := `&{POS [] [] 1 41983-20}`
want := `&{POS [] [] 1 0-41983-20}`
got := fmt.Sprintf("%v", event)
if got != want {
t.Errorf("got %s, want %s", got, want)

Просмотреть файл

@ -32,7 +32,7 @@ func TestKeyRangeFilterPass(t *testing.T) {
Sql: []byte("dml2 /* EMD keyspace_id:2 */"),
},
},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("GoogleMysql", "41983-1")},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("MariaDB", "0-41983-1")},
}
var got string
f := KeyRangeFilterFunc(key.KIT_UINT64, testKeyRange, func(reply *proto.BinlogTransaction) error {
@ -40,7 +40,7 @@ func TestKeyRangeFilterPass(t *testing.T) {
return nil
})
f(&input)
want := `statement: <6, "set1"> statement: <4, "dml2 /* EMD keyspace_id:2 */"> position: "41983-1" `
want := `statement: <6, "set1"> statement: <4, "dml2 /* EMD keyspace_id:2 */"> position: "0-41983-1" `
if want != got {
t.Errorf("want %s, got %s", want, got)
}
@ -57,7 +57,7 @@ func TestKeyRangeFilterSkip(t *testing.T) {
Sql: []byte("dml1 /* EMD keyspace_id:20 */"),
},
},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("GoogleMysql", "41983-1")},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("MariaDB", "0-41983-1")},
}
var got string
f := KeyRangeFilterFunc(key.KIT_UINT64, testKeyRange, func(reply *proto.BinlogTransaction) error {
@ -65,7 +65,7 @@ func TestKeyRangeFilterSkip(t *testing.T) {
return nil
})
f(&input)
want := `position: "41983-1" `
want := `position: "0-41983-1" `
if want != got {
t.Errorf("want %s, got %s", want, got)
}
@ -82,7 +82,7 @@ func TestKeyRangeFilterDDL(t *testing.T) {
Sql: []byte("ddl"),
},
},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("GoogleMysql", "41983-1")},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("MariaDB", "0-41983-1")},
}
var got string
f := KeyRangeFilterFunc(key.KIT_UINT64, testKeyRange, func(reply *proto.BinlogTransaction) error {
@ -90,7 +90,7 @@ func TestKeyRangeFilterDDL(t *testing.T) {
return nil
})
f(&input)
want := `position: "41983-1" `
want := `position: "0-41983-1" `
if want != got {
t.Errorf("want %s, got %s", want, got)
}
@ -113,7 +113,7 @@ func TestKeyRangeFilterMalformed(t *testing.T) {
Sql: []byte("dml1 /* EMD keyspace_id:2a */"),
},
},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("GoogleMysql", "41983-1")},
GTIDField: myproto.GTIDField{Value: myproto.MustParseGTID("MariaDB", "0-41983-1")},
}
var got string
f := KeyRangeFilterFunc(key.KIT_UINT64, testKeyRange, func(reply *proto.BinlogTransaction) error {
@ -121,7 +121,7 @@ func TestKeyRangeFilterMalformed(t *testing.T) {
return nil
})
f(&input)
want := `position: "41983-1" `
want := `position: "0-41983-1" `
if want != got {
t.Errorf("want %s, got %s", want, got)
}

Просмотреть файл

@ -17,7 +17,7 @@ import (
var MycnfPath = "/tmp/my.cnf"
func TestMycnf(t *testing.T) {
os.Setenv("MYSQL_FLAVOR", "GoogleMysql")
os.Setenv("MYSQL_FLAVOR", "MariaDB")
dbaConfig := dbconfigs.DefaultDBConfigs.Dba
appConfig := dbconfigs.DefaultDBConfigs.App.ConnParams
replConfig := dbconfigs.DefaultDBConfigs.Repl

Просмотреть файл

@ -1,325 +0,0 @@
// Copyright 2014, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
import (
"bytes"
"encoding/binary"
"fmt"
"strings"
"time"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/sqldb"
blproto "github.com/youtube/vitess/go/vt/binlog/proto"
"github.com/youtube/vitess/go/vt/mysqlctl/proto"
)
// googleMysql51 is the implementation of MysqlFlavor for google mysql 51
type googleMysql51 struct {
}
const googleMysqlFlavorID = "GoogleMysql"
// VersionMatch implements MysqlFlavor.VersionMatch().
func (*googleMysql51) VersionMatch(version string) bool {
return strings.HasPrefix(version, "5.1") && strings.Contains(strings.ToLower(version), "google")
}
// MasterPosition implements MysqlFlavor.MasterPosition().
//
// The command looks like:
// mysql> SHOW MASTER STATUS\G
// **************************** 1. row ***************************
// File: vt-000001c6-bin.000003
// Position: 106
// Binlog_Do_DB:
// Binlog_Ignore_DB:
// Group_ID:
func (flavor *googleMysql51) MasterPosition(mysqld *Mysqld) (rp proto.ReplicationPosition, err error) {
fields, err := mysqld.fetchSuperQueryMap("SHOW MASTER STATUS")
if err != nil {
return rp, err
}
groupID, ok := fields["Group_ID"]
if !ok {
return rp, fmt.Errorf("this db does not support group id")
}
// Get the server_id that created this group_id.
info, err := mysqld.fetchSuperQueryMap("SHOW BINLOG INFO FOR " + groupID)
if err != nil {
return proto.ReplicationPosition{}, err
}
// Google MySQL does not define a format to describe both a server_id and
// group_id, so we invented one.
pos := info["Server_ID"] + "-" + groupID
return flavor.ParseReplicationPosition(pos)
}
// SlaveStatus implements MysqlFlavor.SlaveStatus().
func (flavor *googleMysql51) SlaveStatus(mysqld *Mysqld) (*proto.ReplicationStatus, error) {
fields, err := mysqld.fetchSuperQueryMap("SHOW SLAVE STATUS")
if err != nil {
return nil, ErrNotSlave
}
status := parseSlaveStatus(fields)
groupID := fields["Exec_Master_Group_ID"]
// Get the server_id that created this group_id.
info, err := mysqld.fetchSuperQueryMap("SHOW BINLOG INFO FOR " + groupID)
if err != nil {
return nil, fmt.Errorf("SlaveStatus can't get server_id for group_id (%v): %v", groupID, err)
}
// Create the fake Google GTID syntax we invented.
pos := info["Server_ID"] + "-" + groupID
status.Position, err = flavor.ParseReplicationPosition(pos)
if err != nil {
return nil, fmt.Errorf("SlaveStatus can't parse Google GTID (%v): %v", pos, err)
}
return status, nil
}
// WaitMasterPos implements MysqlFlavor.WaitMasterPos().
//
// waitTimeout of 0 means wait indefinitely.
//
// Google MySQL doesn't have a function to wait for a GTID. MASTER_POS_WAIT()
// requires a file:pos, which we don't know anymore because we're passing around
// only GTIDs internally now.
//
// We can't ask the local mysqld instance to convert the GTID with BinlogInfo()
// because this instance hasn't seen that GTID yet. For now, we have to poll.
//
// There used to be a function called Mysqld.WaitForMinimumReplicationPosition,
// which was the same as WaitMasterPos except it used polling because it worked
// on GTIDs. Now that WaitMasterPos uses GTIDs too, they've been merged.
func (*googleMysql51) WaitMasterPos(mysqld *Mysqld, targetPos proto.ReplicationPosition, waitTimeout time.Duration) error {
stopTime := time.Now().Add(waitTimeout)
for waitTimeout == 0 || time.Now().Before(stopTime) {
status, err := mysqld.SlaveStatus()
if err != nil {
return err
}
if status.Position.AtLeast(targetPos) {
return nil
}
if !status.SlaveRunning() && waitTimeout == 0 {
return fmt.Errorf("slave not running during WaitMasterPos and no timeout is set, status = %+v", status)
}
log.Infof("WaitMasterPos got position %v, sleeping for 1s waiting for position %v", status.Position, targetPos)
time.Sleep(time.Second)
}
return fmt.Errorf("timed out waiting for position %v", targetPos)
}
// ResetReplicationCommands implements MysqlFlavor.ResetReplicationCommands().
func (*googleMysql51) ResetReplicationCommands() []string {
return []string{
"STOP SLAVE",
"RESET SLAVE",
"RESET MASTER",
"CHANGE MASTER TO MASTER_HOST = ''",
}
}
// PromoteSlaveCommands implements MysqlFlavor.PromoteSlaveCommands().
func (*googleMysql51) PromoteSlaveCommands() []string {
return []string{
"RESET MASTER",
"RESET SLAVE",
"CHANGE MASTER TO MASTER_HOST = ''",
}
}
// StartReplicationCommands implements MysqlFlavor.StartReplicationCommands().
func (*googleMysql51) StartReplicationCommands(params *sqldb.ConnParams, status *proto.ReplicationStatus) ([]string, error) {
// Make SET binlog_group_id command. We have to cast to the Google-specific
// struct to access the fields because there is no canonical printed format to
// represent both a group_id and server_id in Google MySQL.
gtid, ok := status.Position.GTIDSet.(proto.GoogleGTID)
if !ok {
return nil, fmt.Errorf("can't start replication at GTIDSet %#v, expected GoogleGTID", status.Position.GTIDSet)
}
setGroupID := fmt.Sprintf(
"SET binlog_group_id = %d, master_server_id = %d",
gtid.GroupID, gtid.ServerID)
// Make CHANGE MASTER TO command.
args := changeMasterArgs(params, status)
args = append(args, "CONNECT_USING_GROUP_ID")
changeMasterTo := "CHANGE MASTER TO\n " + strings.Join(args, ",\n ")
return []string{
"STOP SLAVE",
"RESET SLAVE",
setGroupID,
changeMasterTo,
"START SLAVE",
}, nil
}
// SetMasterCommands implements MysqlFlavor.SetMasterCommands().
func (*googleMysql51) SetMasterCommands(params *sqldb.ConnParams, masterHost string, masterPort int, masterConnectRetry int) ([]string, error) {
args := changeMasterArgs2(params, masterHost, masterPort, masterConnectRetry)
args = append(args, "CONNECT_USING_GROUP_ID")
changeMasterTo := "CHANGE MASTER TO\n " + strings.Join(args, ",\n ")
return []string{
"STOP SLAVE",
changeMasterTo,
"START SLAVE",
}, nil
}
// ParseGTID implements MysqlFlavor.ParseGTID().
func (*googleMysql51) ParseGTID(s string) (proto.GTID, error) {
return proto.ParseGTID(googleMysqlFlavorID, s)
}
// ParseReplicationPosition implements MysqlFlavor.ParseReplicationPosition().
func (*googleMysql51) ParseReplicationPosition(s string) (proto.ReplicationPosition, error) {
return proto.ParseReplicationPosition(googleMysqlFlavorID, s)
}
// EnableBinlogPlayback implements MysqlFlavor.EnableBinlogPlayback().
func (*googleMysql51) EnableBinlogPlayback(mysqld *Mysqld) error {
// The Google-specific option super_to_set_timestamp is on by default.
// We need to turn it off when we're about to start binlog streamer.
if err := mysqld.ExecuteSuperQuery("SET @@global.super_to_set_timestamp = 0"); err != nil {
log.Errorf("Cannot set super_to_set_timestamp=0: %v", err)
return fmt.Errorf("EnableBinlogPlayback: can't set super_to_timestamp=0: %v", err)
}
log.Info("Successfully set super_to_set_timestamp=0")
return nil
}
// DisableBinlogPlayback implements MysqlFlavor.DisableBinlogPlayback().
func (*googleMysql51) DisableBinlogPlayback(mysqld *Mysqld) error {
// Re-enable super_to_set_timestamp when we're done streaming.
if err := mysqld.ExecuteSuperQuery("SET @@global.super_to_set_timestamp = 1"); err != nil {
log.Warningf("Cannot set super_to_set_timestamp=1: %v", err)
return fmt.Errorf("DisableBinlogPlayback: can't set super_to_timestamp=1: %v", err)
}
log.Info("Successfully set super_to_set_timestamp=1")
return nil
}
// makeBinlogDump2Command builds a buffer containing the data for a Google MySQL
// COM_BINLOG_DUMP2 command.
func makeBinlogDump2Command(flags uint16, slaveID uint32, groupID uint64, sourceServerID uint32) []byte {
var buf bytes.Buffer
buf.Grow(2 + 4 + 8 + 4)
// binlog_flags (2 bytes)
binary.Write(&buf, binary.LittleEndian, flags)
// server_id of slave (4 bytes)
binary.Write(&buf, binary.LittleEndian, slaveID)
// group_id (8 bytes)
binary.Write(&buf, binary.LittleEndian, groupID)
// server_id of the server that generated the group_id (4 bytes)
binary.Write(&buf, binary.LittleEndian, sourceServerID)
return buf.Bytes()
}
// SendBinlogDumpCommand implements MysqlFlavor.SendBinlogDumpCommand().
func (flavor *googleMysql51) SendBinlogDumpCommand(mysqld *Mysqld, conn *SlaveConnection, startPos proto.ReplicationPosition) error {
const (
ComBinlogDump2 = 0x27
BinlogUseGroupID = 0x04
)
gtid, ok := startPos.GTIDSet.(proto.GoogleGTID)
if !ok {
return fmt.Errorf("startPos.GTIDSet = %#v is wrong type, expected GoogleGTID", startPos.GTIDSet)
}
// Build the command.
buf := makeBinlogDump2Command(BinlogUseGroupID, conn.slaveID, gtid.GroupID, gtid.ServerID)
return conn.SendCommand(ComBinlogDump2, buf)
}
// MakeBinlogEvent implements MysqlFlavor.MakeBinlogEvent().
func (*googleMysql51) MakeBinlogEvent(buf []byte) blproto.BinlogEvent {
return NewGoogleBinlogEvent(buf)
}
// googleBinlogEvent wraps a raw packet buffer and provides methods to examine
// it by implementing blproto.BinlogEvent. Some methods are pulled in from
// binlogEvent.
type googleBinlogEvent struct {
binlogEvent
}
// NewGoogleBinlogEvent creates a BinlogEvent from given byte array
func NewGoogleBinlogEvent(buf []byte) blproto.BinlogEvent {
return googleBinlogEvent{binlogEvent: binlogEvent(buf)}
}
// IsGTID implements BinlogEvent.IsGTID().
func (ev googleBinlogEvent) IsGTID() bool {
// Google MySQL doesn't have a GTID_EVENT.
return false
}
// Format implements BinlogEvent.Format().
func (ev googleBinlogEvent) Format() (blproto.BinlogFormat, error) {
f, err := ev.binlogEvent.Format()
if err != nil {
return f, err
}
// Google MySQL extends the header length (normally 19 bytes) by 8 to add
// the group_id to every event.
if f.HeaderLength < 19+8 {
return f, fmt.Errorf("Google MySQL header length = %v, should be >= %v", f.HeaderLength, 19+8)
}
return f, nil
}
// HasGTID implements BinlogEvent.HasGTID().
func (ev googleBinlogEvent) HasGTID(f blproto.BinlogFormat) bool {
// The header is frozen for FORMAT_DESCRIPTION and ROTATE events, since they
// occur before you know the header_length. Therefore, they don't have the
// extended header with a group_id.
if ev.IsFormatDescription() || ev.IsRotate() {
return false
}
// The group_id field is in the header for every other event type, but it's 0
// if that event type doesn't use it.
return ev.groupID() > 0
}
// groupID returns the group_id from the Google MySQL event header.
func (ev googleBinlogEvent) groupID() uint64 {
// Google extended the header (normally 19 bytes) to add an 8 byte group_id.
return binary.LittleEndian.Uint64(ev.Bytes()[19 : 19+8])
}
// GTID implements BinlogEvent.GTID().
func (ev googleBinlogEvent) GTID(f blproto.BinlogFormat) (proto.GTID, error) {
groupID := ev.groupID()
if groupID == 0 {
return nil, fmt.Errorf("invalid group_id 0")
}
return proto.GoogleGTID{ServerID: ev.ServerID(), GroupID: groupID}, nil
}
// StripChecksum implements BinlogEvent.StripChecksum().
func (ev googleBinlogEvent) StripChecksum(f blproto.BinlogFormat) (blproto.BinlogEvent, []byte) {
// Checksums aren't supported in Google MySQL 5.1.
return ev, nil
}
func init() {
registerFlavorBuiltin(googleMysqlFlavorID, &googleMysql51{})
}

Просмотреть файл

@ -1,414 +0,0 @@
// Copyright 2014, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
import (
"reflect"
"testing"
"github.com/youtube/vitess/go/mysql"
"github.com/youtube/vitess/go/sqldb"
blproto "github.com/youtube/vitess/go/vt/binlog/proto"
proto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
)
func TestGoogleMakeBinlogEvent(t *testing.T) {
input := []byte{1, 2, 3}
want := googleBinlogEvent{binlogEvent: binlogEvent([]byte{1, 2, 3})}
if got := (&googleMysql51{}).MakeBinlogEvent(input); !reflect.DeepEqual(got, want) {
t.Errorf("(&googleMysql51{}).MakeBinlogEvent(%#v) = %#v, want %#v", input, got, want)
}
}
func TestGoogleBinlogEventIsGTID(t *testing.T) {
input := googleBinlogEvent{}
want := false
if got := input.IsGTID(); got != want {
t.Errorf("%#v.IsGTID() = %v, want %v", input, got, want)
}
}
func TestGoogleBinlogEventFormat(t *testing.T) {
input := googleBinlogEvent{binlogEvent: binlogEvent(googleFormatEvent)}
want := blproto.BinlogFormat{
FormatVersion: 4,
ServerVersion: "5.1.63-google-log",
HeaderLength: 27,
}
got, err := input.Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if got != want {
t.Errorf("%#v.Format() = %v, want %v", input, got, want)
}
}
func TestGoogleBinlogEventFormatBadHeaderLength(t *testing.T) {
buf := make([]byte, len(googleFormatEvent))
copy(buf, googleFormatEvent)
buf[19+2+50+4] = 12 // mess up the HeaderLength
input := googleBinlogEvent{binlogEvent: binlogEvent(buf)}
want := "header length = 12, should be >= 19"
_, err := input.Format()
if err == nil {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestGoogleBinlogEventFormatBadGoogleHeaderLength(t *testing.T) {
buf := make([]byte, len(googleFormatEvent))
copy(buf, googleFormatEvent)
buf[19+2+50+4] = 19 // mess up the HeaderLength
input := googleBinlogEvent{binlogEvent: binlogEvent(buf)}
want := "Google MySQL header length = 19, should be >= 27"
_, err := input.Format()
if err == nil {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestGoogleBinlogEventHasGTID(t *testing.T) {
f, err := binlogEvent(googleFormatEvent).Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
input := googleBinlogEvent{binlogEvent: binlogEvent(googleQueryEvent)}
want := true
if got := input.HasGTID(f); got != want {
t.Errorf("%#v.HasGTID() = %v, want %v", input, got, want)
}
}
func TestGoogleBinlogEventFormatDescriptionDoesntHaveGTID(t *testing.T) {
f, err := binlogEvent(googleFormatEvent).Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
input := googleBinlogEvent{binlogEvent: binlogEvent(googleFormatEvent)}
want := false
if got := input.HasGTID(f); got != want {
t.Errorf("%#v.HasGTID() = %v, want %v", input, got, want)
}
}
func TestGoogleBinlogEventRotateDoesntHaveGTID(t *testing.T) {
f, err := binlogEvent(googleFormatEvent).Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
input := googleBinlogEvent{binlogEvent: binlogEvent(googleRotateEvent)}
want := false
if got := input.HasGTID(f); got != want {
t.Errorf("%#v.HasGTID() = %v, want %v", input, got, want)
}
}
func TestGoogleBinlogEventDoesntHaveGTID(t *testing.T) {
f, err := binlogEvent(googleFormatEvent).Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
buf := make([]byte, len(googleQueryEvent))
copy(buf, googleFormatEvent)
copy(buf[19:19+8], make([]byte, 8)) // set group_id = 0
input := googleBinlogEvent{binlogEvent: binlogEvent(buf)}
want := false
if got := input.HasGTID(f); got != want {
t.Errorf("%#v.HasGTID() = %v, want %v", input, got, want)
}
}
func TestGoogleBinlogEventGTIDInvalid(t *testing.T) {
f, err := binlogEvent(googleFormatEvent).Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
buf := make([]byte, len(googleQueryEvent))
copy(buf, googleFormatEvent)
copy(buf[19:19+8], make([]byte, 8)) // set group_id = 0
input := googleBinlogEvent{binlogEvent: binlogEvent(buf)}
want := "invalid group_id 0"
_, err = input.GTID(f)
if err == nil {
t.Errorf("expected error, got none")
return
}
if got := err.Error(); got != want {
t.Errorf("wrong error, got %#v, want %#v", got, want)
}
}
func TestGoogleBinlogEventGTID(t *testing.T) {
f, err := binlogEvent(googleFormatEvent).Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
input := googleBinlogEvent{binlogEvent: binlogEvent(googleQueryEvent)}
want := proto.GoogleGTID{ServerID: 62344, GroupID: 0xb}
got, err := input.GTID(f)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if !reflect.DeepEqual(got, want) {
t.Errorf("%#v.GTID() = %#v, want %#v", input, got, want)
}
}
func TestGoogleBinlogEventStripChecksum(t *testing.T) {
f, err := binlogEvent(googleFormatEvent).Format()
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
input := googleBinlogEvent{binlogEvent: binlogEvent(googleQueryEvent)}
want := input
gotEvent, gotChecksum := input.StripChecksum(f)
if !reflect.DeepEqual(gotEvent, want) || gotChecksum != nil {
t.Errorf("%#v.StripChecksum() = (%v, %v), want (%v, nil)", input, gotEvent, gotChecksum, want)
}
}
func TestGoogleStartReplicationCommands(t *testing.T) {
params := &sqldb.ConnParams{
Uname: "username",
Pass: "password",
}
status := &proto.ReplicationStatus{
Position: proto.ReplicationPosition{GTIDSet: proto.GoogleGTID{ServerID: 41983, GroupID: 12345}},
MasterHost: "localhost",
MasterPort: 123,
MasterConnectRetry: 1234,
}
want := []string{
"STOP SLAVE",
"RESET SLAVE",
"SET binlog_group_id = 12345, master_server_id = 41983",
`CHANGE MASTER TO
MASTER_HOST = 'localhost',
MASTER_PORT = 123,
MASTER_USER = 'username',
MASTER_PASSWORD = 'password',
MASTER_CONNECT_RETRY = 1234,
CONNECT_USING_GROUP_ID`,
"START SLAVE",
}
got, err := (&googleMysql51{}).StartReplicationCommands(params, status)
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
if !reflect.DeepEqual(got, want) {
t.Errorf("(&googleMysql51{}).StartReplicationCommands(%#v, %#v) = %#v, want %#v", params, status, got, want)
}
}
func TestGoogleStartReplicationCommandsSSL(t *testing.T) {
params := &sqldb.ConnParams{
Uname: "username",
Pass: "password",
SslCa: "ssl-ca",
SslCaPath: "ssl-ca-path",
SslCert: "ssl-cert",
SslKey: "ssl-key",
}
mysql.EnableSSL(params)
status := &proto.ReplicationStatus{
Position: proto.ReplicationPosition{GTIDSet: proto.GoogleGTID{ServerID: 41983, GroupID: 12345}},
MasterHost: "localhost",
MasterPort: 123,
MasterConnectRetry: 1234,
}
want := []string{
"STOP SLAVE",
"RESET SLAVE",
"SET binlog_group_id = 12345, master_server_id = 41983",
`CHANGE MASTER TO
MASTER_HOST = 'localhost',
MASTER_PORT = 123,
MASTER_USER = 'username',
MASTER_PASSWORD = 'password',
MASTER_CONNECT_RETRY = 1234,
MASTER_SSL = 1,
MASTER_SSL_CA = 'ssl-ca',
MASTER_SSL_CAPATH = 'ssl-ca-path',
MASTER_SSL_CERT = 'ssl-cert',
MASTER_SSL_KEY = 'ssl-key',
CONNECT_USING_GROUP_ID`,
"START SLAVE",
}
got, err := (&googleMysql51{}).StartReplicationCommands(params, status)
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
if !reflect.DeepEqual(got, want) {
t.Errorf("(&googleMysql51{}).StartReplicationCommands(%#v, %#v) = %#v, want %#v", params, status, got, want)
}
}
func TestGoogleSetMasterCommands(t *testing.T) {
params := &sqldb.ConnParams{
Uname: "username",
Pass: "password",
}
masterHost := "localhost"
masterPort := 123
masterConnectRetry := 1234
want := []string{
"STOP SLAVE",
`CHANGE MASTER TO
MASTER_HOST = 'localhost',
MASTER_PORT = 123,
MASTER_USER = 'username',
MASTER_PASSWORD = 'password',
MASTER_CONNECT_RETRY = 1234,
CONNECT_USING_GROUP_ID`,
"START SLAVE",
}
got, err := (&googleMysql51{}).SetMasterCommands(params, masterHost, masterPort, masterConnectRetry)
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
if !reflect.DeepEqual(got, want) {
t.Errorf("(&googleMysql51{}).SetMasterCommands(%#v, %#v, %#v, %#v) = %#v, want %#v", params, masterHost, masterPort, masterConnectRetry, got, want)
}
}
func TestGoogleSetMasterCommandsSSL(t *testing.T) {
params := &sqldb.ConnParams{
Uname: "username",
Pass: "password",
SslCa: "ssl-ca",
SslCaPath: "ssl-ca-path",
SslCert: "ssl-cert",
SslKey: "ssl-key",
}
mysql.EnableSSL(params)
masterHost := "localhost"
masterPort := 123
masterConnectRetry := 1234
want := []string{
"STOP SLAVE",
`CHANGE MASTER TO
MASTER_HOST = 'localhost',
MASTER_PORT = 123,
MASTER_USER = 'username',
MASTER_PASSWORD = 'password',
MASTER_CONNECT_RETRY = 1234,
MASTER_SSL = 1,
MASTER_SSL_CA = 'ssl-ca',
MASTER_SSL_CAPATH = 'ssl-ca-path',
MASTER_SSL_CERT = 'ssl-cert',
MASTER_SSL_KEY = 'ssl-key',
CONNECT_USING_GROUP_ID`,
"START SLAVE",
}
got, err := (&googleMysql51{}).SetMasterCommands(params, masterHost, masterPort, masterConnectRetry)
if err != nil {
t.Errorf("unexpected error: %v", err)
return
}
if !reflect.DeepEqual(got, want) {
t.Errorf("(&googleMysql51{}).SetMasterCommands(%#v, %#v, %#v, %#v) = %#v, want %#v", params, masterHost, masterPort, masterConnectRetry, got, want)
}
}
func TestGoogleParseGTID(t *testing.T) {
input := "123-456"
want := proto.GoogleGTID{ServerID: 123, GroupID: 456}
got, err := (&googleMysql51{}).ParseGTID(input)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if got != want {
t.Errorf("(&googleMysql51{}).ParseGTID(%#v) = %#v, want %#v", input, got, want)
}
}
func TestGoogleParseReplicationPosition(t *testing.T) {
input := "123-456"
want := proto.ReplicationPosition{GTIDSet: proto.GoogleGTID{ServerID: 123, GroupID: 456}}
got, err := (&googleMysql51{}).ParseReplicationPosition(input)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if !got.Equal(want) {
t.Errorf("(&googleMysql51{}).ParseReplicationPosition(%#v) = %#v, want %#v", input, got, want)
}
}
func TestMakeBinlogDump2Command(t *testing.T) {
want := []byte{
// binlog_flags
0xfe, 0xca,
// slave_server_id
0xef, 0xbe, 0xad, 0xde,
// group_id
0x78, 0x56, 0x34, 0x12, 0x78, 0x56, 0x34, 0x12,
// event_server_id
0x21, 0x43, 0x65, 0x87,
}
got := makeBinlogDump2Command(0xcafe, 0xdeadbeef, 0x1234567812345678, 0x87654321)
if !reflect.DeepEqual(got, want) {
t.Errorf("makeBinlogDump2Command() = %#v, want %#v", got, want)
}
}
func TestGooglePromoteSlaveCommands(t *testing.T) {
want := []string{
"RESET MASTER",
"RESET SLAVE",
"CHANGE MASTER TO MASTER_HOST = ''",
}
if got := (&googleMysql51{}).PromoteSlaveCommands(); !reflect.DeepEqual(got, want) {
t.Errorf("(&googleMysql51{}).PromoteSlaveCommands() = %#v, want %#v", got, want)
}
}
func TestGoogleVersionMatch(t *testing.T) {
table := map[string]bool{
"10.0.13-MariaDB-1~precise-log": false,
"5.1.63-google-log": true,
}
for input, want := range table {
if got := (&googleMysql51{}).VersionMatch(input); got != want {
t.Errorf("(&googleMysql51{}).VersionMatch(%#v) = %v, want %v", input, got, want)
}
}
}

Просмотреть файл

@ -1,136 +0,0 @@
// Copyright 2014, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package proto
import (
"fmt"
"strconv"
"strings"
)
const googleMysqlFlavorID = "GoogleMysql"
// parseGoogleGTID is registered as a GTID parser.
func parseGoogleGTID(s string) (GTID, error) {
// Split into parts.
parts := strings.Split(s, "-")
if len(parts) != 2 {
return nil, fmt.Errorf("invalid Google MySQL GTID (%v): expecting ServerID-GroupID", s)
}
server_id, err := strconv.ParseUint(parts[0], 10, 32)
if err != nil {
return nil, fmt.Errorf("invalid Google MySQL server_id (%v): %v", parts[0], err)
}
group_id, err := strconv.ParseUint(parts[1], 10, 64)
if err != nil {
return nil, fmt.Errorf("invalid Google MySQL group_id (%v): %v", parts[1], err)
}
return GoogleGTID{ServerID: uint32(server_id), GroupID: group_id}, nil
}
// parseGoogleGTIDSet is registered as a GTIDSet parser.
func parseGoogleGTIDSet(s string) (GTIDSet, error) {
gtid, err := parseGoogleGTID(s)
if err != nil {
return nil, err
}
return gtid.(GoogleGTID), err
}
// GoogleGTID implements GTID and GTIDSet. In Google MySQL, a single GTID is
// already enough to define the set of all GTIDs that came before it.
type GoogleGTID struct {
// ServerID is the server_id of the server that originally generated the
// transaction.
ServerID uint32
// GroupID is the unique ID of a transaction group.
GroupID uint64
}
// String implements GTID.String(). Google MySQL doesn't define a canonical way
// to represent both a group_id and a server_id together, so we've invented one.
func (gtid GoogleGTID) String() string {
return fmt.Sprintf("%d-%d", gtid.ServerID, gtid.GroupID)
}
// Flavor implements GTID.Flavor().
func (gtid GoogleGTID) Flavor() string {
return googleMysqlFlavorID
}
// Domain implements GTID.SequenceDomain().
func (gtid GoogleGTID) SequenceDomain() string {
return ""
}
// SourceServer implements GTID.SourceServer().
func (gtid GoogleGTID) SourceServer() string {
return strconv.FormatUint(uint64(gtid.ServerID), 10)
}
// SequenceNumber implements GTID.SequenceNumber().
func (gtid GoogleGTID) SequenceNumber() uint64 {
return gtid.GroupID
}
// GTIDSet implements GTID.GTIDSet().
func (gtid GoogleGTID) GTIDSet() GTIDSet {
return gtid
}
// Last implements GTIDSet.Last().
func (gtid GoogleGTID) Last() GTID {
return gtid
}
// ContainsGTID implements GTIDSet.ContainsGTID().
func (gtid GoogleGTID) ContainsGTID(other GTID) bool {
if other == nil {
return true
}
gOther, ok := other.(GoogleGTID)
if !ok {
return false
}
return gtid.GroupID >= gOther.GroupID
}
// Contains implements GTIDSet.Contains().
func (gtid GoogleGTID) Contains(other GTIDSet) bool {
if other == nil {
return true
}
gOther, ok := other.(GoogleGTID)
if !ok {
return false
}
return gtid.GroupID >= gOther.GroupID
}
// Equal implements GTIDSet.Equal().
func (gtid GoogleGTID) Equal(other GTIDSet) bool {
gOther, ok := other.(GoogleGTID)
if !ok {
return false
}
return gtid.GroupID == gOther.GroupID
}
// AddGTID implements GTIDSet.AddGTID().
func (gtid GoogleGTID) AddGTID(other GTID) GTIDSet {
gOther, ok := other.(GoogleGTID)
if !ok || gtid.GroupID >= gOther.GroupID {
return gtid
}
return gOther
}
func init() {
gtidParsers[googleMysqlFlavorID] = parseGoogleGTID
gtidSetParsers[googleMysqlFlavorID] = parseGoogleGTIDSet
}

Просмотреть файл

@ -1,359 +0,0 @@
// Copyright 2014, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package proto
import (
"strings"
"testing"
)
func TestParseGoogleGTID(t *testing.T) {
input := "41983-1758283"
want := GoogleGTID{ServerID: 41983, GroupID: 1758283}
got, err := parseGoogleGTID(input)
if err != nil {
t.Errorf("%v", err)
}
if got.(GoogleGTID) != want {
t.Errorf("parseGoogleGTID(%v) = %v, want %v", input, got, want)
}
}
func TestParseGoogleGTIDSet(t *testing.T) {
input := "41983-1758283"
want := GoogleGTID{ServerID: 41983, GroupID: 1758283}
got, err := parseGoogleGTIDSet(input)
if err != nil {
t.Errorf("%v", err)
}
if got.(GoogleGTID) != want {
t.Errorf("parseGoogleGTIDSet(%#v) = %#v, want %#v", input, got, want)
}
}
func TestParseInvalidGoogleGTIDSet(t *testing.T) {
input := "12-3456d78"
want := "invalid Google MySQL group_id"
_, err := parseGoogleGTIDSet(input)
if err == nil {
t.Errorf("expected error for invalid input (%#v)", input)
return
}
if got := err.Error(); !strings.HasPrefix(got, want) {
t.Errorf("parseGoogleGTIDSet(%#v) error = %#v, want %#v", input, got, want)
}
}
func TestParseInvalidGoogleGTID(t *testing.T) {
input := "12345"
want := "invalid Google MySQL GTID"
_, err := parseGoogleGTID(input)
if err == nil {
t.Errorf("expected error for invalid input (%v)", input)
}
if !strings.HasPrefix(err.Error(), want) {
t.Errorf("wrong error message, got '%v', want '%v'", err, want)
}
}
func TestParseInvalidGoogleServerID(t *testing.T) {
input := "1d3-45"
want := "invalid Google MySQL server_id"
_, err := parseGoogleGTID(input)
if err == nil {
t.Errorf("expected error for invalid input (%v)", input)
}
if !strings.HasPrefix(err.Error(), want) {
t.Errorf("wrong error message, got '%v', want '%v'", err, want)
}
}
func TestParseInvalidGoogleGroupID(t *testing.T) {
input := "1-2d3"
want := "invalid Google MySQL group_id"
_, err := parseGoogleGTID(input)
if err == nil {
t.Errorf("expected error for invalid input (%v)", input)
}
if !strings.HasPrefix(err.Error(), want) {
t.Errorf("wrong error message, got '%v', want '%v'", err, want)
}
}
func TestGoogleGTIDString(t *testing.T) {
input := GoogleGTID{ServerID: 41983, GroupID: 1857273}
want := "41983-1857273"
got := input.String()
if got != want {
t.Errorf("%#v.String() = '%v', want '%v'", input, got, want)
}
}
func TestGoogleGTIDFlavor(t *testing.T) {
input := GoogleGTID{GroupID: 123}
want := "GoogleMysql"
got := input.Flavor()
if got != want {
t.Errorf("%#v.Flavor() = '%v', want '%v'", input, got, want)
}
}
func TestGoogleGTIDSequenceDomain(t *testing.T) {
input := GoogleGTID{ServerID: 41983, GroupID: 1857273}
want := ""
got := input.SequenceDomain()
if got != want {
t.Errorf("%#v.SequenceDomain() = '%v', want '%v'", input, got, want)
}
}
func TestGoogleGTIDSourceServer(t *testing.T) {
input := GoogleGTID{ServerID: 41983, GroupID: 1857273}
want := "41983"
got := input.SourceServer()
if got != want {
t.Errorf("%#v.SourceServer() = '%v', want '%v'", input, got, want)
}
}
func TestGoogleGTIDSequenceNumber(t *testing.T) {
input := GoogleGTID{ServerID: 41983, GroupID: 1857273}
want := uint64(1857273)
got := input.SequenceNumber()
if got != want {
t.Errorf("%#v.SequenceNumber() = %v, want %v", input, got, want)
}
}
func TestGoogleGTIDGTIDSet(t *testing.T) {
input := GoogleGTID{ServerID: 41983, GroupID: 1857273}
want := GTIDSet(input)
got := input.GTIDSet()
if got != want {
t.Errorf("%#v.GTIDSet() = %#v, want %#v", input, got, want)
}
}
func TestGoogleGTIDLast(t *testing.T) {
input := GoogleGTID{ServerID: 41983, GroupID: 1857273}
want := GTID(input)
got := input.Last()
if got != want {
t.Errorf("%#v.Last() = %#v, want %#v", input, got, want)
}
}
func TestGoogleGTIDContainsLess(t *testing.T) {
input1 := GoogleGTID{GroupID: 12345}
input2 := GoogleGTID{GroupID: 54321}
want := false
if got := input1.Contains(input2); got != want {
t.Errorf("%#v.Contains(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsGreater(t *testing.T) {
input1 := GoogleGTID{GroupID: 54321}
input2 := GoogleGTID{GroupID: 12345}
want := true
if got := input1.Contains(input2); got != want {
t.Errorf("%#v.Contains(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsEqual(t *testing.T) {
input1 := GoogleGTID{GroupID: 12345}
input2 := GoogleGTID{GroupID: 12345}
want := true
if got := input1.Contains(input2); got != want {
t.Errorf("%#v.Contains(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsWrongType(t *testing.T) {
input1 := GoogleGTID{GroupID: 123}
input2 := fakeGTID{}
want := false
if got := input1.Contains(input2); got != want {
t.Errorf("%#v.Contains(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsNil(t *testing.T) {
input1 := GoogleGTID{GroupID: 123}
input2 := GTIDSet(nil)
want := true
if got := input1.Contains(input2); got != want {
t.Errorf("%#v.Contains(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsGTIDLess(t *testing.T) {
input1 := GoogleGTID{GroupID: 12345}
input2 := GoogleGTID{GroupID: 54321}
want := false
if got := input1.ContainsGTID(input2); got != want {
t.Errorf("%#v.ContainsGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsGTIDGreater(t *testing.T) {
input1 := GoogleGTID{GroupID: 54321}
input2 := GoogleGTID{GroupID: 12345}
want := true
if got := input1.ContainsGTID(input2); got != want {
t.Errorf("%#v.ContainsGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsGTIDEqual(t *testing.T) {
input1 := GoogleGTID{GroupID: 12345}
input2 := GoogleGTID{GroupID: 12345}
want := true
if got := input1.ContainsGTID(input2); got != want {
t.Errorf("%#v.ContainsGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsGTIDWrongType(t *testing.T) {
input1 := GoogleGTID{GroupID: 123}
input2 := fakeGTID{}
want := false
if got := input1.ContainsGTID(input2); got != want {
t.Errorf("%#v.ContainsGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDContainsGTIDNil(t *testing.T) {
input1 := GoogleGTID{GroupID: 123}
input2 := GTID(nil)
want := true
if got := input1.ContainsGTID(input2); got != want {
t.Errorf("%#v.ContainsGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDEqual(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := GoogleGTID{GroupID: 41234}
want := true
if got := input1.Equal(input2); got != want {
t.Errorf("%#v.Equal(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDNotEqual(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := GoogleGTID{GroupID: 51234}
want := false
if got := input1.Equal(input2); got != want {
t.Errorf("%#v.Equal(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDEqualWrongType(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := fakeGTID{}
want := false
if got := input1.Equal(input2); got != want {
t.Errorf("%#v.Equal(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDEqualNil(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := GTIDSet(nil)
want := false
if got := input1.Equal(input2); got != want {
t.Errorf("%#v.Equal(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDAddGTIDEqual(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := GoogleGTID{GroupID: 41234}
want := GoogleGTID{GroupID: 41234}
if got := input1.AddGTID(input2); got != want {
t.Errorf("%#v.AddGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDAddGTIDGreater(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := GoogleGTID{GroupID: 51234}
want := GoogleGTID{GroupID: 51234}
if got := input1.AddGTID(input2); got != want {
t.Errorf("%#v.AddGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDAddGTIDLess(t *testing.T) {
input1 := GoogleGTID{GroupID: 51234}
input2 := GoogleGTID{GroupID: 41234}
want := GoogleGTID{GroupID: 51234}
if got := input1.AddGTID(input2); got != want {
t.Errorf("%#v.AddGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDAddGTIDWrongType(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := fakeGTID{}
want := input1
if got := input1.AddGTID(input2); got != want {
t.Errorf("%#v.AddGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDAddGTIDNil(t *testing.T) {
input1 := GoogleGTID{GroupID: 41234}
input2 := GTID(nil)
want := input1
if got := input1.AddGTID(input2); got != want {
t.Errorf("%#v.AddGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}
func TestGoogleGTIDAddGTIDDifferentServer(t *testing.T) {
input1 := GoogleGTID{ServerID: 1, GroupID: 41234}
input2 := GoogleGTID{ServerID: 2, GroupID: 51234}
want := GoogleGTID{ServerID: 2, GroupID: 51234}
if got := input1.AddGTID(input2); got != want {
t.Errorf("%#v.AddGTID(%#v) = %v, want %v", input1, input2, got, want)
}
}

Просмотреть файл

@ -293,39 +293,6 @@ func (mysqld *Mysqld) SetMasterCommands(masterHost string, masterPort int, maste
return flavor.SetMasterCommands(&params, masterHost, masterPort, masterConnectRetry)
}
/*
mysql> SHOW BINLOG INFO FOR 5\G
*************************** 1. row ***************************
Log_name: vt-0000041983-bin.000001
Pos: 1194
Server_ID: 41983
*/
// BinlogInfo returns the filename and position for a Google MySQL group_id.
// This command only exists in Google MySQL.
func (mysqld *Mysqld) BinlogInfo(pos proto.ReplicationPosition) (fileName string, filePos uint, err error) {
if pos.IsZero() {
return fileName, filePos, fmt.Errorf("input position for BinlogInfo is uninitialized")
}
// Extract the group_id from the GoogleGTID. We can't just use String() on the
// ReplicationPosition, because that includes the server_id.
gtid, ok := pos.GTIDSet.(proto.GoogleGTID)
if !ok {
return "", 0, fmt.Errorf("Non-Google GTID in BinlogInfo(%#v), which is only supported on Google MySQL", pos)
}
info, err := mysqld.fetchSuperQueryMap(fmt.Sprintf("SHOW BINLOG INFO FOR %v", gtid.GroupID))
if err != nil {
return "", 0, err
}
fileName = info["Log_name"]
temp, err := strconv.ParseUint(info["Pos"], 10, 32)
if err != nil {
return fileName, filePos, err
}
filePos = uint(temp)
return fileName, filePos, err
}
// WaitForSlave waits for a slave if its lag is larger than given maxLag
func (mysqld *Mysqld) WaitForSlave(maxLag int) (err error) {
// FIXME(msolomon) verify that slave started based on show slave status;

Просмотреть файл

@ -10,6 +10,7 @@ import (
"fmt"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/mysql"
"github.com/youtube/vitess/go/pools"
"github.com/youtube/vitess/go/sqldb"
"github.com/youtube/vitess/go/sync2"
@ -105,8 +106,8 @@ func (sc *SlaveConnection) StartBinlogDump(startPos proto.ReplicationPosition) (
buf, err = sc.Conn.ReadPacket()
if err != nil {
if sqlErr, ok := err.(*sqldb.SqlError); ok && sqlErr.Number() == 2013 {
// errno 2013 = Lost connection to MySQL server during query
if sqlErr, ok := err.(*sqldb.SqlError); ok && sqlErr.Number() == mysql.ErrServerLost {
// ErrServerLost = Lost connection to MySQL server during query
// This is not necessarily an error. It could just be that we closed
// the connection from outside.
log.Infof("connection closed during binlog stream (possibly intentional): %v", err)

Просмотреть файл

@ -32,6 +32,7 @@ import (
"golang.org/x/net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/history"
"github.com/youtube/vitess/go/jscfg"
"github.com/youtube/vitess/go/netutil"
@ -41,6 +42,7 @@ import (
"github.com/youtube/vitess/go/vt/health"
"github.com/youtube/vitess/go/vt/mysqlctl"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/tabletmanager/events"
"github.com/youtube/vitess/go/vt/tabletserver"
"github.com/youtube/vitess/go/vt/topo"
)
@ -222,7 +224,16 @@ func (agent *ActionAgent) updateState(ctx context.Context, oldTablet *topo.Table
newTablet := agent._tablet.Tablet
agent.mutex.Unlock()
log.Infof("Running tablet callback because: %v", reason)
return agent.changeCallback(ctx, oldTablet, newTablet)
if err := agent.changeCallback(ctx, oldTablet, newTablet); err != nil {
return err
}
event.Dispatch(&events.StateChange{
OldTablet: *oldTablet,
NewTablet: *newTablet,
Reason: reason,
})
return nil
}
func (agent *ActionAgent) readTablet(ctx context.Context) (*topo.TabletInfo, error) {

Просмотреть файл

@ -0,0 +1,23 @@
// Copyright 2015, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
// Package events contains event structs used by the tabletmanager package.
package events
import (
"github.com/youtube/vitess/go/vt/topo"
)
// StateChange is an event that describes state changes in the tablet as seen
// by the TabletManager. It is triggered after the tablet has processed a state
// change, which might have been initiated internally, or when the tablet
// noticed that an external process modified its topo record.
type StateChange struct {
// OldTablet is the topo record of the tablet before the change.
OldTablet topo.Tablet
// NewTablet is the topo record representing the current state.
NewTablet topo.Tablet
// Reason is an optional string that describes the source of the change.
Reason string
}

Просмотреть файл

@ -42,9 +42,10 @@ func NewCachePool(
name string,
rowCacheConfig RowCacheConfig,
idleTimeout time.Duration,
statsURL string) *CachePool {
statsURL string,
enablePublishStats bool) *CachePool {
cp := &CachePool{name: name, idleTimeout: idleTimeout, statsURL: statsURL}
if name != "" {
if name != "" && enablePublishStats {
cp.memcacheStats = NewMemcacheStats(
rowCacheConfig.StatsPrefix+name, 10*time.Second, enableMain,
func(key string) string {

Просмотреть файл

@ -6,9 +6,11 @@ package tabletserver
import (
"fmt"
"io/ioutil"
"math/rand"
"net/http"
"net/http/httptest"
"regexp"
"testing"
"time"
@ -20,7 +22,7 @@ import (
func TestCachePoolWithEmptyBinary(t *testing.T) {
fakecacheservice.Register()
fakesqldb.Register()
cachePool := newTestCachePool(RowCacheConfig{})
cachePool := newTestCachePool(RowCacheConfig{}, false)
cachePool.Close()
}
@ -31,7 +33,7 @@ func TestCachePool(t *testing.T) {
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, false)
if !cachePool.IsClosed() {
t.Fatalf("cache pool is not closed")
}
@ -52,7 +54,7 @@ func TestCachePoolOpenTwice(t *testing.T) {
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, false)
cachePool.Open()
defer cachePool.Close()
defer func() {
@ -70,7 +72,7 @@ func TestCachePoolOpenWithEmptyBinary(t *testing.T) {
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, false)
defer func() {
if e := recover(); e == nil {
t.Fatalf("open a cache pool with empty rowCacheConfig.Binary should panic")
@ -88,7 +90,7 @@ func TestCachePoolOpenWithInvalidBinary(t *testing.T) {
Binary: "invalid_binary",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, false)
defer func() {
if e := recover(); e == nil {
t.Fatalf("open a cache pool with an invalid rowCacheConfig.Binary should panic")
@ -105,7 +107,7 @@ func TestCachePoolState(t *testing.T) {
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, true)
idleTimeout := 1 * time.Second
cachePool.idleTimeout = idleTimeout
cachePool.Open()
@ -141,7 +143,7 @@ func TestCachePoolStateWithoutOpen(t *testing.T) {
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, false)
idleTimeout := 1 * time.Second
cachePool.idleTimeout = idleTimeout
if cachePool.StatsJSON() != "{}" {
@ -175,7 +177,7 @@ func TestCachePoolGetFailedBecauseCachePoolIsClosed(t *testing.T) {
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, false)
idleTimeout := 1 * time.Second
cachePool.idleTimeout = idleTimeout
ctx := context.Background()
@ -188,25 +190,76 @@ func TestCachePoolGetFailedBecauseCachePoolIsClosed(t *testing.T) {
}
func TestCachePoolStatsURL(t *testing.T) {
fakecacheservice.Register()
cache := fakecacheservice.Register()
fakesqldb.Register()
rowCacheConfig := RowCacheConfig{
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig)
cachePool := newTestCachePool(rowCacheConfig, false)
idleTimeout := 1 * time.Second
cachePool.idleTimeout = idleTimeout
cachePool.Open()
request, _ := http.NewRequest("GET", fmt.Sprintf("%sstats", cachePool.statsURL), nil)
response := httptest.NewRecorder()
cachePool.ServeHTTP(response, request)
// any memcache calls should fail
cache.EnableCacheServiceError()
response = httptest.NewRecorder()
cachePool.ServeHTTP(response, request)
cache.DisableCacheServiceError()
cachePool.Close()
response = httptest.NewRecorder()
cachePool.ServeHTTP(response, request)
body, _ := ioutil.ReadAll(response.Body)
matcher := regexp.MustCompile("closed")
if !matcher.Match(body) {
t.Fatalf("stats page should contain 'closed', but got %s", string(body))
}
}
func TestCachePoolMemcacheStatsFail(t *testing.T) {
cache := fakecacheservice.Register()
fakesqldb.Register()
rowCacheConfig := RowCacheConfig{
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig, true)
idleTimeout := 1 * time.Second
cachePool.idleTimeout = idleTimeout
cachePool.Open()
defer cachePool.Close()
request, _ := http.NewRequest("GET", cachePool.statsURL, nil)
response := httptest.NewRecorder()
cachePool.ServeHTTP(response, request)
memcacheStatsBefore := internalErrors.Counts()["MemcacheStats"]
// any memcache calls should fail
cache.EnableCacheServiceError()
cachePool.memcacheStats.update()
memcacheStatsAfter := internalErrors.Counts()["MemcacheStats"]
if memcacheStatsAfter <= memcacheStatsBefore {
t.Fatalf("memcache stats should cause an internal error")
}
}
func newTestCachePool(rowcacheConfig RowCacheConfig) *CachePool {
func TestCachePoolFailToStartBecauseCacheServiceWasDown(t *testing.T) {
cache := fakecacheservice.Register()
fakesqldb.Register()
testUtils := &testUtils{}
rowCacheConfig := RowCacheConfig{
Binary: "ls",
Connections: 100,
}
cachePool := newTestCachePool(rowCacheConfig, false)
idleTimeout := 1 * time.Second
cachePool.idleTimeout = idleTimeout
// any memcache calls should fail
cache.EnableCacheServiceError()
defer testUtils.checkTabletErrorWithRecover(t, ErrFatal, "can't communicate with cache service")
cachePool.Open()
}
func newTestCachePool(rowcacheConfig RowCacheConfig, enablePublishStats bool) *CachePool {
randID := rand.Int63()
name := fmt.Sprintf("TestCachePool-%d-", randID)
statsURL := fmt.Sprintf("/debug/cache-%d", randID)
return NewCachePool(name, rowcacheConfig, 1*time.Second, statsURL)
statsURL := fmt.Sprintf("/debug/cache-%d/", randID)
return NewCachePool(name, rowcacheConfig, 1*time.Second, statsURL, enablePublishStats)
}

Просмотреть файл

@ -41,7 +41,8 @@ type ConnPool struct {
func NewConnPool(
name string,
capacity int,
idleTimeout time.Duration) *ConnPool {
idleTimeout time.Duration,
enablePublishStats bool) *ConnPool {
cp := &ConnPool{
capacity: capacity,
idleTimeout: idleTimeout,
@ -50,12 +51,14 @@ func NewConnPool(
if name == "" {
return cp
}
stats.Publish(name+"Capacity", stats.IntFunc(cp.Capacity))
stats.Publish(name+"Available", stats.IntFunc(cp.Available))
stats.Publish(name+"MaxCap", stats.IntFunc(cp.MaxCap))
stats.Publish(name+"WaitCount", stats.IntFunc(cp.WaitCount))
stats.Publish(name+"WaitTime", stats.DurationFunc(cp.WaitTime))
stats.Publish(name+"IdleTimeout", stats.DurationFunc(cp.IdleTimeout))
if enablePublishStats {
stats.Publish(name+"Capacity", stats.IntFunc(cp.Capacity))
stats.Publish(name+"Available", stats.IntFunc(cp.Available))
stats.Publish(name+"MaxCap", stats.IntFunc(cp.MaxCap))
stats.Publish(name+"WaitCount", stats.IntFunc(cp.WaitCount))
stats.Publish(name+"WaitTime", stats.DurationFunc(cp.WaitTime))
stats.Publish(name+"IdleTimeout", stats.DurationFunc(cp.IdleTimeout))
}
return cp
}

Просмотреть файл

@ -42,7 +42,7 @@ func TestConnectivity(t *testing.T) {
killStats = stats.NewCounters("TestKills")
internalErrors = stats.NewCounters("TestInternalErrors")
mysqlStats = stats.NewTimings("TestMySQLStats")
pool := NewConnPool("p1", 1, 30*time.Second)
pool := NewConnPool("p1", 1, 30*time.Second, false)
pool.Open(appParams, dbaParams)
conn, err := pool.Get(ctx)
@ -78,7 +78,7 @@ func TestConnectivity(t *testing.T) {
newctx, cancel = withTimeout(ctx, 2*time.Millisecond)
_, err = conn.Exec(newctx, "select sleep(1) from dual", 1000, true)
cancel()
lostConn := "error: Lost connection to MySQL server during query (errno 2013) during query: select sleep(1) from dual"
lostConn := "error: the query was killed either because it timed out or was canceled: Lost connection to MySQL server during query (errno 2013) during query: select sleep(1) from dual"
if err == nil || err.Error() != lostConn {
t.Errorf("got: %v, want %s", err, lostConn)
}

Просмотреть файл

@ -12,8 +12,11 @@ import (
"time"
cs "github.com/youtube/vitess/go/cacheservice"
"github.com/youtube/vitess/go/sync2"
)
var errCacheService = "cacheservice error"
// FakeCacheService is a fake implementation of CacheService
type FakeCacheService struct {
cache *Cache
@ -21,8 +24,9 @@ type FakeCacheService struct {
// Cache is a cache like data structure.
type Cache struct {
data map[string]*cs.Result
mu sync.Mutex
mu sync.Mutex
data map[string]*cs.Result
enableCacheServiceError sync2.AtomicInt32
}
// Set sets a key and associated value to the cache.
@ -61,6 +65,16 @@ func (cache *Cache) Clear() {
cache.data = make(map[string]*cs.Result)
}
// EnableCacheServiceError makes cache service return error.
func (cache *Cache) EnableCacheServiceError() {
cache.enableCacheServiceError.Set(1)
}
// DisableCacheServiceError makes cache service back to normal.
func (cache *Cache) DisableCacheServiceError() {
cache.enableCacheServiceError.Set(0)
}
// NewFakeCacheService creates a FakeCacheService
func NewFakeCacheService(cache *Cache) *FakeCacheService {
return &FakeCacheService{
@ -70,6 +84,9 @@ func NewFakeCacheService(cache *Cache) *FakeCacheService {
// Get returns cached data for given keys.
func (service *FakeCacheService) Get(keys ...string) ([]cs.Result, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return nil, fmt.Errorf(errCacheService)
}
results := make([]cs.Result, 0, len(keys))
for _, key := range keys {
if val, ok := service.cache.Get(key); ok {
@ -83,6 +100,9 @@ func (service *FakeCacheService) Get(keys ...string) ([]cs.Result, error) {
// for using with CAS. Gets returns a CAS identifier with the item. If
// the item's CAS value has changed since you Gets'ed it, it will not be stored.
func (service *FakeCacheService) Gets(keys ...string) ([]cs.Result, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return nil, fmt.Errorf(errCacheService)
}
results := make([]cs.Result, 0, len(keys))
for _, key := range keys {
if val, ok := service.cache.Get(key); ok {
@ -96,6 +116,9 @@ func (service *FakeCacheService) Gets(keys ...string) ([]cs.Result, error) {
// Set set the value with specified cache key.
func (service *FakeCacheService) Set(key string, flags uint16, timeout uint64, value []byte) (bool, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return false, fmt.Errorf(errCacheService)
}
service.cache.Set(key, &cs.Result{
Key: key,
Value: value,
@ -107,6 +130,9 @@ func (service *FakeCacheService) Set(key string, flags uint16, timeout uint64, v
// Add store the value only if it does not already exist.
func (service *FakeCacheService) Add(key string, flags uint16, timeout uint64, value []byte) (bool, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return false, fmt.Errorf(errCacheService)
}
if _, ok := service.cache.Get(key); ok {
return false, nil
}
@ -122,6 +148,9 @@ func (service *FakeCacheService) Add(key string, flags uint16, timeout uint64, v
// Replace replaces the value, only if the value already exists,
// for the specified cache key.
func (service *FakeCacheService) Replace(key string, flags uint16, timeout uint64, value []byte) (bool, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return false, fmt.Errorf(errCacheService)
}
result, ok := service.cache.Get(key)
if !ok {
return false, nil
@ -134,6 +163,9 @@ func (service *FakeCacheService) Replace(key string, flags uint16, timeout uint6
// Append appends the value after the last bytes in an existing item.
func (service *FakeCacheService) Append(key string, flags uint16, timeout uint64, value []byte) (bool, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return false, fmt.Errorf(errCacheService)
}
result, ok := service.cache.Get(key)
if !ok {
return false, nil
@ -146,6 +178,9 @@ func (service *FakeCacheService) Append(key string, flags uint16, timeout uint64
// Prepend prepends the value before existing value.
func (service *FakeCacheService) Prepend(key string, flags uint16, timeout uint64, value []byte) (bool, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return false, fmt.Errorf(errCacheService)
}
result, ok := service.cache.Get(key)
if !ok {
return false, nil
@ -158,6 +193,9 @@ func (service *FakeCacheService) Prepend(key string, flags uint16, timeout uint6
// Cas stores the value only if no one else has updated the data since you read it last.
func (service *FakeCacheService) Cas(key string, flags uint16, timeout uint64, value []byte, cas uint64) (bool, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return false, fmt.Errorf(errCacheService)
}
result, ok := service.cache.Get(key)
if !ok || result.Cas != cas {
return false, nil
@ -171,18 +209,27 @@ func (service *FakeCacheService) Cas(key string, flags uint16, timeout uint64, v
// Delete delete the value for the specified cache key.
func (service *FakeCacheService) Delete(key string) (bool, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return false, fmt.Errorf(errCacheService)
}
service.cache.Delete(key)
return true, nil
}
// FlushAll purges the entire cache.
func (service *FakeCacheService) FlushAll() error {
if service.cache.enableCacheServiceError.Get() == 1 {
return fmt.Errorf(errCacheService)
}
service.cache.Clear()
return nil
}
// Stats returns a list of basic stats.
func (service *FakeCacheService) Stats(key string) ([]byte, error) {
if service.cache.enableCacheServiceError.Get() == 1 {
return nil, fmt.Errorf(errCacheService)
}
return []byte{}, nil
}

Просмотреть файл

@ -122,3 +122,50 @@ func TestFakeCacheService(t *testing.T) {
service.Stats("")
service.Close()
}
func TestFakeCacheServiceError(t *testing.T) {
service := NewFakeCacheService(&Cache{data: make(map[string]*cs.Result)})
service.cache.EnableCacheServiceError()
key1 := "key1"
_, err := service.Set(key1, 0, 0, []byte("test"))
checkCacheServiceError(t, err)
_, err = service.Get(key1)
checkCacheServiceError(t, err)
_, err = service.Gets(key1)
checkCacheServiceError(t, err)
_, err = service.Cas(key1, 0, 0, []byte("test2"), 0)
checkCacheServiceError(t, err)
_, err = service.Add(key1, 0, 0, []byte("test3"))
checkCacheServiceError(t, err)
_, err = service.Replace("unknownKey", 0, 0, []byte("test4"))
checkCacheServiceError(t, err)
_, err = service.Append("unknownKey", 0, 0, []byte("test5"))
checkCacheServiceError(t, err)
_, err = service.Prepend("unknownKey", 0, 0, []byte("test5"))
checkCacheServiceError(t, err)
_, err = service.Prepend(key1, 0, 0, []byte("test5"))
checkCacheServiceError(t, err)
_, err = service.Delete(key1)
checkCacheServiceError(t, err)
err = service.FlushAll()
checkCacheServiceError(t, err)
_, err = service.Stats("")
checkCacheServiceError(t, err)
service.cache.DisableCacheServiceError()
ok, err := service.Set(key1, 0, 0, []byte("test"))
if !ok || err != nil {
t.Fatalf("set should succeed")
}
results, err := service.Get(key1)
if !reflect.DeepEqual(results[0].Value, []byte("test")) {
t.Fatalf("expect to get value: test, but get: %s", string(results[0].Value))
}
service.Close()
}
func checkCacheServiceError(t *testing.T, err error) {
if err.Error() != errCacheService {
t.Fatalf("should get cacheservice error")
}
}

Просмотреть файл

@ -7,6 +7,8 @@ package tabletserver
import (
"bytes"
"net/http"
"strconv"
"time"
)
func startHTMLTable(w http.ResponseWriter) {
@ -128,3 +130,28 @@ func wrappable(in string) string {
}
return buf.String()
}
func adjustValue(val int, lower int, upper int) int {
if val < lower {
return lower
} else if val > upper {
return upper
}
return val
}
func parseTimeoutLimitParams(req *http.Request) (time.Duration, int) {
timeout := 10
limit := 300
if ts, ok := req.URL.Query()["timeout"]; ok {
if t, err := strconv.Atoi(ts[0]); err == nil {
timeout = adjustValue(t, 0, 60)
}
}
if l, ok := req.URL.Query()["limit"]; ok {
if lim, err := strconv.Atoi(l[0]); err == nil {
limit = adjustValue(lim, 1, 200000)
}
}
return time.Duration(timeout) * time.Second, limit
}

Просмотреть файл

@ -127,6 +127,7 @@ func NewQueryEngine(config Config) *QueryEngine {
},
time.Duration(config.SchemaReloadTime*1e9),
time.Duration(config.IdleTimeout*1e9),
config.EnablePublishStats,
)
mysqlStats = stats.NewTimings(config.StatsPrefix + "Mysql")
@ -137,16 +138,19 @@ func NewQueryEngine(config Config) *QueryEngine {
config.RowCache,
time.Duration(config.IdleTimeout*1e9),
config.DebugURLPrefix+"/memcache/",
config.EnablePublishStats,
)
qe.connPool = NewConnPool(
config.PoolNamePrefix+"ConnPool",
config.PoolSize,
time.Duration(config.IdleTimeout*1e9),
config.EnablePublishStats,
)
qe.streamConnPool = NewConnPool(
config.PoolNamePrefix+"StreamConnPool",
config.StreamPoolSize,
time.Duration(config.IdleTimeout*1e9),
config.EnablePublishStats,
)
// Services
@ -157,6 +161,7 @@ func NewQueryEngine(config Config) *QueryEngine {
time.Duration(config.TransactionTimeout*1e9),
time.Duration(config.TxPoolTimeout*1e9),
time.Duration(config.IdleTimeout*1e9),
config.EnablePublishStats,
)
qe.consolidator = sync2.NewConsolidator()
http.Handle(config.DebugURLPrefix+"/consolidations", qe.consolidator)
@ -178,10 +183,6 @@ func NewQueryEngine(config Config) *QueryEngine {
qe.accessCheckerLogger = logutil.NewThrottledLogger("accessChecker", 1*time.Second)
// Stats
stats.Publish(config.StatsPrefix+"MaxResultSize", stats.IntFunc(qe.maxResultSize.Get))
stats.Publish(config.StatsPrefix+"MaxDMLRows", stats.IntFunc(qe.maxDMLRows.Get))
stats.Publish(config.StatsPrefix+"StreamBufferSize", stats.IntFunc(qe.streamBufferSize.Get))
stats.Publish(config.StatsPrefix+"QueryTimeout", stats.DurationFunc(qe.queryTimeout.Get))
queryStats = stats.NewTimings(config.StatsPrefix + "Queries")
qpsRates = stats.NewRates(config.StatsPrefix+"QPS", queryStats, 15, 60*time.Second)
waitStats = stats.NewTimings(config.StatsPrefix + "Waits")
@ -190,11 +191,17 @@ func NewQueryEngine(config Config) *QueryEngine {
errorStats = stats.NewCounters(config.StatsPrefix + "Errors")
internalErrors = stats.NewCounters(config.StatsPrefix + "InternalErrors")
resultStats = stats.NewHistogram(config.StatsPrefix+"Results", resultBuckets)
stats.Publish(config.StatsPrefix+"RowcacheSpotCheckRatio", stats.FloatFunc(func() float64 {
return float64(qe.spotCheckFreq.Get()) / spotCheckMultiplier
}))
spotCheckCount = stats.NewInt(config.StatsPrefix + "RowcacheSpotCheckCount")
if config.EnablePublishStats {
stats.Publish(config.StatsPrefix+"MaxResultSize", stats.IntFunc(qe.maxResultSize.Get))
stats.Publish(config.StatsPrefix+"MaxDMLRows", stats.IntFunc(qe.maxDMLRows.Get))
stats.Publish(config.StatsPrefix+"StreamBufferSize", stats.IntFunc(qe.streamBufferSize.Get))
stats.Publish(config.StatsPrefix+"QueryTimeout", stats.DurationFunc(qe.queryTimeout.Get))
stats.Publish(config.StatsPrefix+"RowcacheSpotCheckRatio", stats.FloatFunc(func() float64 {
return float64(qe.spotCheckFreq.Get()) / spotCheckMultiplier
}))
}
return qe
}

Просмотреть файл

@ -732,6 +732,7 @@ func newTestQueryExecutor(sql string, ctx context.Context, flags executorFlags)
config.PoolSize = 100
config.TransactionCap = 100
config.SpotCheckRatio = 1.0
config.EnablePublishStats = false
if flags&enableStrict > 0 {
config.StrictMode = true
} else {
@ -747,8 +748,10 @@ func newTestQueryExecutor(sql string, ctx context.Context, flags executorFlags)
config.StrictTableAcl = false
}
sqlQuery := NewSqlQuery(config)
testUtils := newTestUtils()
txID := int64(0)
dbconfigs := newTestDBConfigs()
dbconfigs := testUtils.newDBConfigs()
if flags&enableRowCache > 0 {
dbconfigs.App.EnableRowcache = true
} else {
@ -758,7 +761,7 @@ func newTestQueryExecutor(sql string, ctx context.Context, flags executorFlags)
if flags&enableSchemaOverrides > 0 {
schemaOverrides = getTestTableSchemaOverrides()
}
sqlQuery.allowQueries(&dbconfigs, schemaOverrides, newMysqld(&dbconfigs))
sqlQuery.allowQueries(&dbconfigs, schemaOverrides, testUtils.newMysqld(&dbconfigs))
if flags&enableTx > 0 {
session := proto.Session{
SessionId: sqlQuery.sessionID,

Просмотреть файл

@ -32,32 +32,33 @@ var (
)
func init() {
flag.IntVar(&qsConfig.PoolSize, "queryserver-config-pool-size", DefaultQsConfig.PoolSize, "query server pool size")
flag.IntVar(&qsConfig.StreamPoolSize, "queryserver-config-stream-pool-size", DefaultQsConfig.StreamPoolSize, "query server stream pool size")
flag.IntVar(&qsConfig.TransactionCap, "queryserver-config-transaction-cap", DefaultQsConfig.TransactionCap, "query server transaction cap")
flag.Float64Var(&qsConfig.TransactionTimeout, "queryserver-config-transaction-timeout", DefaultQsConfig.TransactionTimeout, "query server transaction timeout")
flag.IntVar(&qsConfig.MaxResultSize, "queryserver-config-max-result-size", DefaultQsConfig.MaxResultSize, "query server max result size")
flag.IntVar(&qsConfig.MaxDMLRows, "queryserver-config-max-dml-rows", DefaultQsConfig.MaxDMLRows, "query server max dml rows per statement")
flag.IntVar(&qsConfig.StreamBufferSize, "queryserver-config-stream-buffer-size", DefaultQsConfig.StreamBufferSize, "query server stream buffer size")
flag.IntVar(&qsConfig.QueryCacheSize, "queryserver-config-query-cache-size", DefaultQsConfig.QueryCacheSize, "query server query cache size")
flag.Float64Var(&qsConfig.SchemaReloadTime, "queryserver-config-schema-reload-time", DefaultQsConfig.SchemaReloadTime, "query server schema reload time")
flag.Float64Var(&qsConfig.QueryTimeout, "queryserver-config-query-timeout", DefaultQsConfig.QueryTimeout, "query server query timeout")
flag.Float64Var(&qsConfig.TxPoolTimeout, "queryserver-config-txpool-timeout", DefaultQsConfig.TxPoolTimeout, "query server transaction pool timeout")
flag.Float64Var(&qsConfig.IdleTimeout, "queryserver-config-idle-timeout", DefaultQsConfig.IdleTimeout, "query server idle timeout")
flag.Float64Var(&qsConfig.SpotCheckRatio, "queryserver-config-spot-check-ratio", DefaultQsConfig.SpotCheckRatio, "query server rowcache spot check frequency")
flag.IntVar(&qsConfig.PoolSize, "queryserver-config-pool-size", DefaultQsConfig.PoolSize, "query server connection pool size, connection pool is used by regular queries (non streaming, not in a transaction)")
flag.IntVar(&qsConfig.StreamPoolSize, "queryserver-config-stream-pool-size", DefaultQsConfig.StreamPoolSize, "query server stream pool size, stream pool is used by stream queries: queries that return results to client in a streaming fashion")
flag.IntVar(&qsConfig.TransactionCap, "queryserver-config-transaction-cap", DefaultQsConfig.TransactionCap, "query server transaction cap is the maximum number of transactions allowed to happen at any given point of a time for a single vttablet. E.g. by setting transaction cap to 100, there are at most 100 transactions will be processed by a vttablet and the 101th transaction will be blocked (and fail if it cannot get connection within specified timeout)")
flag.Float64Var(&qsConfig.TransactionTimeout, "queryserver-config-transaction-timeout", DefaultQsConfig.TransactionTimeout, "query server transaction timeout (in seconds), a transaction will be killed if it takes longer than this value")
flag.IntVar(&qsConfig.MaxResultSize, "queryserver-config-max-result-size", DefaultQsConfig.MaxResultSize, "query server max result size, maximum number of rows allowed to return from vttablet for non-streaming queries.")
flag.IntVar(&qsConfig.MaxDMLRows, "queryserver-config-max-dml-rows", DefaultQsConfig.MaxDMLRows, "query server max dml rows per statement, maximum number of rows allowed to return at a time for an upadte or delete with either 1) an equality where clauses on primary keys, or 2) a subselect statement. For update and delete statements in above two categories, vttablet will split the original query into multiple small queries based on this configuration value. ")
flag.IntVar(&qsConfig.StreamBufferSize, "queryserver-config-stream-buffer-size", DefaultQsConfig.StreamBufferSize, "query server stream buffer size, the maximum number of bytes sent from vttablet for each stream call.")
flag.IntVar(&qsConfig.QueryCacheSize, "queryserver-config-query-cache-size", DefaultQsConfig.QueryCacheSize, "query server query cache size, maximum number of queries to be cached. vttablet analyzes every incoming query and generate a query plan, these plans are being cached in a lru cache. This config controls the capacity of the lru cache.")
flag.Float64Var(&qsConfig.SchemaReloadTime, "queryserver-config-schema-reload-time", DefaultQsConfig.SchemaReloadTime, "query server schema reload time, how often vttablet reloads schemas from underlying MySQL instance in seconds. vttablet keeps table schemas in its own memory and periodically refreshes it from MySQL. This config controls the reload time.")
flag.Float64Var(&qsConfig.QueryTimeout, "queryserver-config-query-timeout", DefaultQsConfig.QueryTimeout, "query server query timeout (in seconds), this is the query timeout in vttablet side. If a query takes more than this timeout, it will be killed.")
flag.Float64Var(&qsConfig.TxPoolTimeout, "queryserver-config-txpool-timeout", DefaultQsConfig.TxPoolTimeout, "query server transaction pool timeout, it is how long vttablet waits if tx pool is full")
flag.Float64Var(&qsConfig.IdleTimeout, "queryserver-config-idle-timeout", DefaultQsConfig.IdleTimeout, "query server idle timeout (in seconds), vttablet manages various mysql connection pools. This config means if a connection has not been used in given idle timeout, this connection will be removed from pool. This effectively manages number of connection objects and optimize the pool performance.")
flag.Float64Var(&qsConfig.SpotCheckRatio, "queryserver-config-spot-check-ratio", DefaultQsConfig.SpotCheckRatio, "query server rowcache spot check frequency (in [0, 1]), if rowcache is enabled, this value determines how often a row retrieved from the rowcache is spot-checked against MySQL.")
flag.BoolVar(&qsConfig.StrictMode, "queryserver-config-strict-mode", DefaultQsConfig.StrictMode, "allow only predictable DMLs and enforces MySQL's STRICT_TRANS_TABLES")
flag.BoolVar(&qsConfig.StrictTableAcl, "queryserver-config-strict-table-acl", DefaultQsConfig.StrictTableAcl, "only allow queries that pass table acl checks")
flag.BoolVar(&qsConfig.TerseErrors, "queryserver-config-terse-errors", DefaultQsConfig.TerseErrors, "prevent bind vars from escaping in returned errors")
flag.StringVar(&qsConfig.RowCache.Binary, "rowcache-bin", DefaultQsConfig.RowCache.Binary, "rowcache binary file")
flag.BoolVar(&qsConfig.EnablePublishStats, "queryserver-config-enable-publish-stats", DefaultQsConfig.EnablePublishStats, "set this flag to true makes queryservice publish monitoring stats")
flag.StringVar(&qsConfig.RowCache.Binary, "rowcache-bin", DefaultQsConfig.RowCache.Binary, "rowcache binary file, vttablet launches a memcached if rowcache is enabled. This config specifies the location of the memcache binary.")
flag.IntVar(&qsConfig.RowCache.Memory, "rowcache-memory", DefaultQsConfig.RowCache.Memory, "rowcache max memory usage in MB")
flag.StringVar(&qsConfig.RowCache.Socket, "rowcache-socket", DefaultQsConfig.RowCache.Socket, "socket filename hint: a unique filename will be generated based on this input")
flag.IntVar(&qsConfig.RowCache.Connections, "rowcache-connections", DefaultQsConfig.RowCache.Connections, "rowcache max simultaneous connections")
flag.IntVar(&qsConfig.RowCache.Threads, "rowcache-threads", DefaultQsConfig.RowCache.Threads, "rowcache number of threads")
flag.BoolVar(&qsConfig.RowCache.LockPaged, "rowcache-lock-paged", DefaultQsConfig.RowCache.LockPaged, "whether rowcache locks down paged memory")
flag.StringVar(&qsConfig.RowCache.StatsPrefix, "rowcache-stats-prefix", DefaultQsConfig.RowCache.StatsPrefix, "rowcache stats prefix")
flag.StringVar(&qsConfig.RowCache.StatsPrefix, "rowcache-stats-prefix", DefaultQsConfig.RowCache.StatsPrefix, "rowcache stats prefix, rowcache will export various metrics and this config specifies the metric prefix")
flag.StringVar(&qsConfig.StatsPrefix, "stats-prefix", DefaultQsConfig.StatsPrefix, "prefix for variable names exported via expvar")
flag.StringVar(&qsConfig.DebugURLPrefix, "debug-url-prefix", DefaultQsConfig.DebugURLPrefix, "debug url prefix")
flag.StringVar(&qsConfig.PoolNamePrefix, "pool-name-prefix", DefaultQsConfig.PoolNamePrefix, "pool name prefix")
flag.StringVar(&qsConfig.DebugURLPrefix, "debug-url-prefix", DefaultQsConfig.DebugURLPrefix, "debug url prefix, vttablet will report various system debug pages and this config controls the prefix of these debug urls")
flag.StringVar(&qsConfig.PoolNamePrefix, "pool-name-prefix", DefaultQsConfig.PoolNamePrefix, "pool name prefix, vttablet has several pools and each of them has a name. This config specifies the prefix of these pool names")
}
// RowCacheConfig encapsulates the configuration for RowCache
@ -114,6 +115,7 @@ type Config struct {
StrictMode bool
StrictTableAcl bool
TerseErrors bool
EnablePublishStats bool
StatsPrefix string
DebugURLPrefix string
PoolNamePrefix string
@ -145,6 +147,7 @@ var DefaultQsConfig = Config{
StrictMode: true,
StrictTableAcl: false,
TerseErrors: false,
EnablePublishStats: true,
StatsPrefix: "",
DebugURLPrefix: "/debug",
PoolNamePrefix: "",

Просмотреть файл

@ -14,6 +14,7 @@ import (
log "github.com/golang/glog"
"github.com/youtube/vitess/go/acl"
"github.com/youtube/vitess/go/streamlog"
)
var (
@ -71,25 +72,28 @@ var (
)
func init() {
http.HandleFunc("/querylogz", querylogzHandler)
http.HandleFunc("/querylogz", func(w http.ResponseWriter, r *http.Request) {
querylogzHandler(SqlQueryLogger, w, r)
})
}
// querylogzHandler serves a human readable snapshot of the
// current query log.
func querylogzHandler(w http.ResponseWriter, r *http.Request) {
func querylogzHandler(streamLogger *streamlog.StreamLogger, w http.ResponseWriter, r *http.Request) {
if err := acl.CheckAccessHTTP(r, acl.DEBUGGING); err != nil {
acl.SendError(w, err)
return
}
ch := SqlQueryLogger.Subscribe("querylogz")
defer SqlQueryLogger.Unsubscribe(ch)
timeout, limit := parseTimeoutLimitParams(r)
ch := streamLogger.Subscribe("querylogz")
defer streamLogger.Unsubscribe(ch)
startHTMLTable(w)
defer endHTMLTable(w)
w.Write(querylogzHeader)
tmr := time.NewTimer(10 * time.Second)
tmr := time.NewTimer(timeout)
defer tmr.Stop()
for i := 0; i < 300; i++ {
for i := 0; i < limit; i++ {
select {
case out := <-ch:
stats, ok := out.(*SQLQueryStats)

Просмотреть файл

@ -0,0 +1,142 @@
// Copyright 2015, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletserver
import (
"io/ioutil"
"net/http"
"net/http/httptest"
"regexp"
"strings"
"testing"
"time"
"github.com/youtube/vitess/go/streamlog"
"github.com/youtube/vitess/go/vt/tabletserver/planbuilder"
"golang.org/x/net/context"
)
func TestQuerylogzHandlerInvalidSqlQueryStats(t *testing.T) {
req, _ := http.NewRequest("GET", "/querylogz?timeout=0&limit=10", nil)
response := httptest.NewRecorder()
testLogger := streamlog.New("TestLogger", 100)
testLogger.Send("test msg")
querylogzHandler(testLogger, response, req)
if !strings.Contains(response.Body.String(), "error") {
t.Fatalf("should show an error page for an non SqlQueryStats")
}
}
func TestQuerylogzHandler(t *testing.T) {
req, _ := http.NewRequest("GET", "/querylogz?timeout=0&limit=10", nil)
logStats := newSqlQueryStats("Execute", context.Background())
logStats.PlanType = planbuilder.PLAN_PASS_SELECT.String()
logStats.OriginalSql = "select name from test_table limit 1000"
logStats.RowsAffected = 1000
logStats.NumberOfQueries = 1
logStats.StartTime = time.Unix(123456789, 0)
logStats.MysqlResponseTime = 1 * time.Millisecond
logStats.WaitingForConnection = 10 * time.Nanosecond
logStats.CacheHits = 17
logStats.CacheAbsent = 5
logStats.CacheMisses = 2
logStats.CacheInvalidations = 3
logStats.TransactionID = 131
testLogger := streamlog.New("TestLogger", 100)
// fast query
fastQueryPattern := []string{
`<td>Execute</td>`,
`<td></td>`,
`<td>Nov 29 13:33:09.000000</td>`,
`<td>Nov 29 13:33:09.001000</td>`,
`<td>0.001</td>`,
`<td>0.001</td>`,
`<td>1e-08</td>`,
`<td>PASS_SELECT</td>`,
`<td>select name from test_table limit 1000</td>`,
`<td>1</td>`,
`<td>none</td>`,
`<td>1000</td>`,
`<td>0</td>`,
`<td>17</td>`,
`<td>2</td>`,
`<td>5</td>`,
`<td>3</td>`,
`<td>131</td>`,
`<td></td>`,
}
logStats.EndTime = logStats.StartTime.Add(1 * time.Millisecond)
testLogger.Send(logStats)
response := httptest.NewRecorder()
querylogzHandler(testLogger, response, req)
body, _ := ioutil.ReadAll(response.Body)
checkQuerylogzHasStats(t, fastQueryPattern, logStats, body)
// medium query
mediumQueryPattern := []string{
`<td>Execute</td>`,
`<td></td>`,
`<td>Nov 29 13:33:09.000000</td>`,
`<td>Nov 29 13:33:09.020000</td>`,
`<td>0.02</td>`,
`<td>0.001</td>`,
`<td>1e-08</td>`,
`<td>PASS_SELECT</td>`,
`<td>select name from test_table limit 1000</td>`,
`<td>1</td>`,
`<td>none</td>`,
`<td>1000</td>`,
`<td>0</td>`,
`<td>17</td>`,
`<td>2</td>`,
`<td>5</td>`,
`<td>3</td>`,
`<td>131</td>`,
`<td></td>`,
}
logStats.EndTime = logStats.StartTime.Add(20 * time.Millisecond)
testLogger.Send(logStats)
response = httptest.NewRecorder()
querylogzHandler(testLogger, response, req)
body, _ = ioutil.ReadAll(response.Body)
checkQuerylogzHasStats(t, mediumQueryPattern, logStats, body)
// slow query
slowQueryPattern := []string{
`<td>Execute</td>`,
`<td></td>`,
`<td>Nov 29 13:33:09.000000</td>`,
`<td>Nov 29 13:33:09.500000</td>`,
`<td>0.5</td>`,
`<td>0.001</td>`,
`<td>1e-08</td>`,
`<td>PASS_SELECT</td>`,
`<td>select name from test_table limit 1000</td>`,
`<td>1</td>`,
`<td>none</td>`,
`<td>1000</td>`,
`<td>0</td>`,
`<td>17</td>`,
`<td>2</td>`,
`<td>5</td>`,
`<td>3</td>`,
`<td>131</td>`,
`<td></td>`,
}
logStats.EndTime = logStats.StartTime.Add(500 * time.Millisecond)
testLogger.Send(logStats)
querylogzHandler(testLogger, response, req)
body, _ = ioutil.ReadAll(response.Body)
checkQuerylogzHasStats(t, slowQueryPattern, logStats, body)
}
func checkQuerylogzHasStats(t *testing.T, pattern []string, logStats *SQLQueryStats, page []byte) {
matcher := regexp.MustCompile(strings.Join(pattern, `\s*`))
if !matcher.Match(page) {
t.Fatalf("querylogz page does not contain stats: %v, page: %s", logStats, string(page))
}
}

Просмотреть файл

@ -19,7 +19,7 @@ import (
func TestQueryzHandler(t *testing.T) {
resp := httptest.NewRecorder()
req, _ := http.NewRequest("GET", "/schemaz", nil)
schemaInfo := newTestSchemaInfo(100, 10*time.Second, 10*time.Second)
schemaInfo := newTestSchemaInfo(100, 10*time.Second, 10*time.Second, false)
plan1 := &ExecPlan{
ExecPlan: &planbuilder.ExecPlan{
@ -104,6 +104,6 @@ func TestQueryzHandler(t *testing.T) {
func checkQueryzHasPlan(t *testing.T, planPattern []string, plan *ExecPlan, page []byte) {
matcher := regexp.MustCompile(strings.Join(planPattern, `\s*`))
if !matcher.Match(page) {
t.Fatalf("schemaz page does not contain plan: %v, page: %s", plan, string(page))
t.Fatalf("queryz page does not contain plan: %v, page: %s", plan, string(page))
}
}

Просмотреть файл

@ -117,27 +117,30 @@ func NewSchemaInfo(
statsPrefix string,
endpoints map[string]string,
reloadTime time.Duration,
idleTimeout time.Duration) *SchemaInfo {
idleTimeout time.Duration,
enablePublishStats bool) *SchemaInfo {
si := &SchemaInfo{
queries: cache.NewLRUCache(int64(queryCacheSize)),
connPool: NewConnPool("", 2, idleTimeout),
connPool: NewConnPool("", 2, idleTimeout, enablePublishStats),
ticks: timer.NewTimer(reloadTime),
endpoints: endpoints,
reloadTime: reloadTime,
}
stats.Publish(statsPrefix+"QueryCacheLength", stats.IntFunc(si.queries.Length))
stats.Publish(statsPrefix+"QueryCacheSize", stats.IntFunc(si.queries.Size))
stats.Publish(statsPrefix+"QueryCacheCapacity", stats.IntFunc(si.queries.Capacity))
stats.Publish(statsPrefix+"QueryCacheOldest", stats.StringFunc(func() string {
return fmt.Sprintf("%v", si.queries.Oldest())
}))
stats.Publish(statsPrefix+"SchemaReloadTime", stats.DurationFunc(si.ticks.Interval))
_ = stats.NewMultiCountersFunc(statsPrefix+"RowcacheStats", []string{"Table", "Stats"}, si.getRowcacheStats)
_ = stats.NewMultiCountersFunc(statsPrefix+"RowcacheInvalidations", []string{"Table"}, si.getRowcacheInvalidations)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryCounts", []string{"Table", "Plan"}, si.getQueryCount)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryTimesNs", []string{"Table", "Plan"}, si.getQueryTime)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryRowCounts", []string{"Table", "Plan"}, si.getQueryRowCount)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryErrorCounts", []string{"Table", "Plan"}, si.getQueryErrorCount)
if enablePublishStats {
stats.Publish(statsPrefix+"QueryCacheLength", stats.IntFunc(si.queries.Length))
stats.Publish(statsPrefix+"QueryCacheSize", stats.IntFunc(si.queries.Size))
stats.Publish(statsPrefix+"QueryCacheCapacity", stats.IntFunc(si.queries.Capacity))
stats.Publish(statsPrefix+"QueryCacheOldest", stats.StringFunc(func() string {
return fmt.Sprintf("%v", si.queries.Oldest())
}))
stats.Publish(statsPrefix+"SchemaReloadTime", stats.DurationFunc(si.ticks.Interval))
_ = stats.NewMultiCountersFunc(statsPrefix+"RowcacheStats", []string{"Table", "Stats"}, si.getRowcacheStats)
_ = stats.NewMultiCountersFunc(statsPrefix+"RowcacheInvalidations", []string{"Table"}, si.getRowcacheInvalidations)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryCounts", []string{"Table", "Plan"}, si.getQueryCount)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryTimesNs", []string{"Table", "Plan"}, si.getQueryTime)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryRowCounts", []string{"Table", "Plan"}, si.getQueryRowCount)
_ = stats.NewMultiCountersFunc(statsPrefix+"QueryErrorCounts", []string{"Table", "Plan"}, si.getQueryErrorCount)
}
for _, ep := range endpoints {
http.Handle(ep, si)
}

Просмотреть файл

@ -29,10 +29,10 @@ func TestSchemaInfoStrictMode(t *testing.T) {
for query, result := range getSchemaInfoBaseTestQueries() {
db.AddQuery(query, result)
}
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
defer handleAndVerifyTabletError(
@ -54,10 +54,10 @@ func TestSchemaInfoOpenFailedDueToMissMySQLTime(t *testing.T) {
[]sqltypes.Value{sqltypes.MakeString([]byte("1427325875"))},
},
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
defer handleAndVerifyTabletError(
@ -78,10 +78,10 @@ func TestSchemaInfoOpenFailedDueToIncorrectMysqlRowNum(t *testing.T) {
nil,
},
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
defer handleAndVerifyTabletError(
@ -102,10 +102,10 @@ func TestSchemaInfoOpenFailedDueToInvalidTimeFormat(t *testing.T) {
[]sqltypes.Value{sqltypes.MakeString([]byte("invalid_time"))},
},
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
defer handleAndVerifyTabletError(
@ -126,10 +126,10 @@ func TestSchemaInfoOpenFailedDueToExecErr(t *testing.T) {
// this will cause connection failed to execute baseShowTables query
RowsAffected: math.MaxUint64,
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
defer handleAndVerifyTabletError(
@ -156,10 +156,10 @@ func TestSchemaInfoOpenFailedDueToTableInfoErr(t *testing.T) {
// this will cause NewTableInfo error
RowsAffected: math.MaxUint64,
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
defer handleAndVerifyTabletError(
@ -176,10 +176,10 @@ func TestSchemaInfoOpenWithSchemaOverride(t *testing.T) {
for query, result := range getSchemaInfoTestSupportedQueries() {
db.AddQuery(query, result)
}
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second)
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
schemaOverrides := getSchemaInfoTestSchemaOverride()
@ -206,10 +206,10 @@ func TestSchemaInfoReload(t *testing.T) {
db.AddQuery(query, result)
}
idleTimeout := 10 * time.Second
schemaInfo := newTestSchemaInfo(10, 10*time.Second, idleTimeout)
schemaInfo := newTestSchemaInfo(10, 10*time.Second, idleTimeout, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
// test cache type RW
@ -289,10 +289,10 @@ func TestSchemaInfoCreateOrUpdateTableFailedDuetoExecErr(t *testing.T) {
RowsAffected: math.MaxUint64,
Rows: [][]sqltypes.Value{createTestTableDescribe("pk")},
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
defer handleAndVerifyTabletError(
@ -317,10 +317,10 @@ func TestSchemaInfoCreateOrUpdateTable(t *testing.T) {
RowsAffected: 1,
Rows: [][]sqltypes.Value{createTestTableDescribe("pk")},
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
schemaInfo.Open(&appParams, &dbaParams, getSchemaInfoTestSchemaOverride(), cachePool, false)
@ -340,10 +340,10 @@ func TestSchemaInfoDropTable(t *testing.T) {
RowsAffected: 1,
Rows: [][]sqltypes.Value{createTestTableDescribe("pk")},
})
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
schemaInfo.Open(&appParams, &dbaParams, getSchemaInfoTestSchemaOverride(), cachePool, false)
@ -365,10 +365,10 @@ func TestSchemaInfoGetPlanPanicDuetoEmptyQuery(t *testing.T) {
for query, result := range getSchemaInfoTestSupportedQueries() {
db.AddQuery(query, result)
}
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second)
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
schemaOverrides := getSchemaInfoTestSchemaOverride()
@ -392,10 +392,10 @@ func TestSchemaInfoQueryCacheFailDueToInvalidCacheSize(t *testing.T) {
for query, result := range getSchemaInfoTestSupportedQueries() {
db.AddQuery(query, result)
}
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second)
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
schemaOverrides := getSchemaInfoTestSchemaOverride()
@ -416,10 +416,10 @@ func TestSchemaInfoQueryCache(t *testing.T) {
for query, result := range getSchemaInfoTestSupportedQueries() {
db.AddQuery(query, result)
}
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second)
schemaInfo := newTestSchemaInfo(10, 10*time.Second, 10*time.Second, true)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(true)
cachePool.Open()
defer cachePool.Close()
schemaOverrides := getSchemaInfoTestSchemaOverride()
@ -452,10 +452,10 @@ func TestSchemaInfoExportVars(t *testing.T) {
for query, result := range getSchemaInfoTestSupportedQueries() {
db.AddQuery(query, result)
}
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, true)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(true)
cachePool.Open()
defer cachePool.Close()
schemaInfo.Open(&appParams, &dbaParams, []SchemaOverride{}, cachePool, true)
@ -471,10 +471,10 @@ func TestSchemaInfoStatsURL(t *testing.T) {
for query, result := range getSchemaInfoTestSupportedQueries() {
db.AddQuery(query, result)
}
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second)
schemaInfo := newTestSchemaInfo(10, 1*time.Second, 1*time.Second, false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
cachePool := newTestSchemaInfoCachePool()
cachePool := newTestSchemaInfoCachePool(false)
cachePool.Open()
defer cachePool.Close()
schemaInfo.Open(&appParams, &dbaParams, []SchemaOverride{}, cachePool, true)
@ -506,7 +506,7 @@ func TestSchemaInfoStatsURL(t *testing.T) {
schemaInfo.ServeHTTP(response, request)
}
func newTestSchemaInfoCachePool() *CachePool {
func newTestSchemaInfoCachePool(enablePublishStats bool) *CachePool {
rowCacheConfig := RowCacheConfig{
Binary: "ls",
Connections: 100,
@ -514,7 +514,7 @@ func newTestSchemaInfoCachePool() *CachePool {
randID := rand.Int63()
name := fmt.Sprintf("TestCachePool-%d-", randID)
statsURL := fmt.Sprintf("/debug/cache-%d", randID)
return NewCachePool(name, rowCacheConfig, 1*time.Second, statsURL)
return NewCachePool(name, rowCacheConfig, 1*time.Second, statsURL, enablePublishStats)
}
func getSchemaInfoBaseTestQueries() map[string]*mproto.QueryResult {

Просмотреть файл

@ -91,13 +91,15 @@ func NewSqlQuery(config Config) *SqlQuery {
config: config,
}
sq.qe = NewQueryEngine(config)
stats.Publish(config.StatsPrefix+"TabletState", stats.IntFunc(func() int64 {
sq.mu.Lock()
state := sq.state
sq.mu.Unlock()
return state
}))
stats.Publish(config.StatsPrefix+"TabletStateName", stats.StringFunc(sq.GetState))
if config.EnablePublishStats {
stats.Publish(config.StatsPrefix+"TabletState", stats.IntFunc(func() int64 {
sq.mu.Lock()
state := sq.state
sq.mu.Unlock()
return state
}))
stats.Publish(config.StatsPrefix+"TabletStateName", stats.StringFunc(sq.GetState))
}
return sq
}
@ -270,7 +272,7 @@ func (sq *SqlQuery) Begin(ctx context.Context, session *proto.Session, txInfo *p
if err = sq.startRequest(session.SessionId, false, false); err != nil {
return err
}
ctx, cancel := withTimeout(ctx, sq.qe.txPool.poolTimeout.Get())
ctx, cancel := withTimeout(ctx, sq.qe.txPool.PoolTimeout())
defer func() {
queryStats.Record("BEGIN", time.Now())
cancel()
@ -335,12 +337,8 @@ func (sq *SqlQuery) handleExecError(query *proto.Query, err *error, logStats *SQ
internalErrors.Add("Panic", 1)
return
}
if sq.config.TerseErrors {
if terr.SqlError == 0 {
*err = terr
} else {
*err = fmt.Errorf("%s(errno %d) during query: %s", terr.Prefix(), terr.SqlError, query.Sql)
}
if sq.config.TerseErrors && terr.SqlError != 0 {
*err = fmt.Errorf("%s(errno %d) during query: %s", terr.Prefix(), terr.SqlError, query.Sql)
} else {
*err = terr
}

Просмотреть файл

@ -5,16 +5,14 @@
package tabletserver
import (
"fmt"
"expvar"
"math/rand"
"strconv"
"testing"
"time"
mproto "github.com/youtube/vitess/go/mysql/proto"
"github.com/youtube/vitess/go/sqldb"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/dbconfigs"
"github.com/youtube/vitess/go/vt/mysqlctl"
"github.com/youtube/vitess/go/vt/tabletserver/fakesqldb"
"github.com/youtube/vitess/go/vt/tabletserver/proto"
"golang.org/x/net/context"
@ -23,11 +21,12 @@ import (
func TestSqlQueryAllowQueriesFailBadConn(t *testing.T) {
db := setUpSqlQueryTest()
db.EnableConnFail()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
checkSqlQueryState(t, sqlQuery, "NOT_SERVING")
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err == nil {
t.Fatalf("SqlQuery.allowQueries should fail")
}
@ -36,15 +35,16 @@ func TestSqlQueryAllowQueriesFailBadConn(t *testing.T) {
func TestSqlQueryAllowQueriesFailStrictModeConflictWithRowCache(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
// disable strict mode
config.StrictMode = false
sqlQuery := NewSqlQuery(config)
checkSqlQueryState(t, sqlQuery, "NOT_SERVING")
dbconfigs := newTestDBConfigs()
dbconfigs := testUtils.newDBConfigs()
// enable rowcache
dbconfigs.App.EnableRowcache = true
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err == nil {
t.Fatalf("SqlQuery.allowQueries should fail because strict mode is disabled while rowcache is enabled.")
}
@ -53,18 +53,19 @@ func TestSqlQueryAllowQueriesFailStrictModeConflictWithRowCache(t *testing.T) {
func TestSqlQueryAllowQueries(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
checkSqlQueryState(t, sqlQuery, "NOT_SERVING")
dbconfigs := newTestDBConfigs()
dbconfigs := testUtils.newDBConfigs()
sqlQuery.setState(StateServing)
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
sqlQuery.disallowQueries()
if err != nil {
t.Fatalf("SqlQuery.allowQueries should success, but get error: %v", err)
}
sqlQuery.setState(StateShuttingTx)
err = sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
err = sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err == nil {
t.Fatalf("SqlQuery.allowQueries should fail")
}
@ -73,10 +74,11 @@ func TestSqlQueryAllowQueries(t *testing.T) {
func TestSqlQueryCheckMysql(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
defer sqlQuery.disallowQueries()
if err != nil {
t.Fatalf("SqlQuery.allowQueries should success but get error: %v", err)
@ -88,10 +90,11 @@ func TestSqlQueryCheckMysql(t *testing.T) {
func TestSqlQueryCheckMysqlFailInvalidConn(t *testing.T) {
db := setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
defer sqlQuery.disallowQueries()
if err != nil {
t.Fatalf("SqlQuery.allowQueries should success but get error: %v", err)
@ -105,12 +108,13 @@ func TestSqlQueryCheckMysqlFailInvalidConn(t *testing.T) {
func TestSqlQueryCheckMysqlFailUninitializedQueryEngine(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
dbconfigs := testUtils.newDBConfigs()
// this causes QueryEngine not being initialized properly
sqlQuery.setState(StateServing)
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
defer sqlQuery.disallowQueries()
if err != nil {
t.Fatalf("SqlQuery.allowQueries should success but get error: %v", err)
@ -123,7 +127,9 @@ func TestSqlQueryCheckMysqlFailUninitializedQueryEngine(t *testing.T) {
func TestSqlQueryCheckMysqlInNotServingState(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
config.EnablePublishStats = true
sqlQuery := NewSqlQuery(config)
// sqlquery start request fail because we are in StateNotServing;
// however, checkMySQL should return true. Here, we always assume
@ -131,19 +137,31 @@ func TestSqlQueryCheckMysqlInNotServingState(t *testing.T) {
if !sqlQuery.checkMySQL() {
t.Fatalf("checkMySQL should return true")
}
tabletState := expvar.Get(config.StatsPrefix + "TabletState")
if tabletState == nil {
t.Fatalf("%sTabletState should be exposed", config.StatsPrefix)
}
varzState, err := strconv.Atoi(tabletState.String())
if err != nil {
t.Fatalf("invalid state reported by expvar, should be a valid state code, but got: %s", tabletState.String())
}
if varzState != StateNotServing {
t.Fatalf("queryservice should be in NOT_SERVING state, but exposed varz reports: %s", stateName[varzState])
}
}
func TestSqlQueryGetSessionId(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
if err := sqlQuery.GetSessionId(nil, nil); err == nil {
t.Fatalf("call GetSessionId should get an error")
}
keyspace := "test_keyspace"
shard := "0"
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -179,10 +197,11 @@ func TestSqlQueryGetSessionId(t *testing.T) {
func TestSqlQueryCommandFailUnMatchedSessionId(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -272,6 +291,7 @@ func TestSqlQueryCommandFailUnMatchedSessionId(t *testing.T) {
func TestSqlQueryCommitTransaciton(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
// sql that will be executed in this test
executeSql := "select * from test_table limit 1000"
executeSqlResult := &mproto.QueryResult{
@ -281,10 +301,10 @@ func TestSqlQueryCommitTransaciton(t *testing.T) {
},
}
db.AddQuery(executeSql, executeSqlResult)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -316,6 +336,7 @@ func TestSqlQueryCommitTransaciton(t *testing.T) {
func TestSqlQueryRollback(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
// sql that will be executed in this test
executeSql := "select * from test_table limit 1000"
executeSqlResult := &mproto.QueryResult{
@ -325,11 +346,10 @@ func TestSqlQueryRollback(t *testing.T) {
},
}
db.AddQuery(executeSql, executeSqlResult)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -361,6 +381,7 @@ func TestSqlQueryRollback(t *testing.T) {
func TestSqlQueryStreamExecute(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
// sql that will be executed in this test
executeSql := "select * from test_table limit 1000"
executeSqlResult := &mproto.QueryResult{
@ -371,11 +392,10 @@ func TestSqlQueryStreamExecute(t *testing.T) {
}
db.AddQuery(executeSql, executeSqlResult)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -412,18 +432,17 @@ func TestSqlQueryStreamExecute(t *testing.T) {
func TestSqlQueryExecuteBatch(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
sql := "insert into test_table values (1, 2)"
sqlResult := &mproto.QueryResult{}
expanedSql := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */"
db.AddQuery(sql, sqlResult)
db.AddQuery(expanedSql, sqlResult)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -462,10 +481,11 @@ func TestSqlQueryExecuteBatch(t *testing.T) {
func TestSqlQueryExecuteBatchFailEmptyQueryList(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -485,12 +505,13 @@ func TestSqlQueryExecuteBatchFailEmptyQueryList(t *testing.T) {
func TestSqlQueryExecuteBatchBeginFail(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
// make "begin" query fail
db.AddRejectedQuery("begin")
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -518,12 +539,13 @@ func TestSqlQueryExecuteBatchBeginFail(t *testing.T) {
func TestSqlQueryExecuteBatchCommitFail(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
// make "commit" query fail
db.AddRejectedQuery("commit")
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -556,6 +578,7 @@ func TestSqlQueryExecuteBatchCommitFail(t *testing.T) {
func TestSqlQueryExecuteBatchSqlExecFailInTransaction(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
sql := "insert into test_table values (1, 2)"
sqlResult := &mproto.QueryResult{}
expanedSql := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */"
@ -567,11 +590,10 @@ func TestSqlQueryExecuteBatchSqlExecFailInTransaction(t *testing.T) {
db.AddRejectedQuery(sql)
db.AddRejectedQuery(expanedSql)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -618,6 +640,7 @@ func TestSqlQueryExecuteBatchSqlExecFailInTransaction(t *testing.T) {
func TestSqlQueryExecuteBatchFailBeginWithoutCommit(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
sql := "insert into test_table values (1, 2)"
sqlResult := &mproto.QueryResult{}
expanedSql := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */"
@ -625,11 +648,10 @@ func TestSqlQueryExecuteBatchFailBeginWithoutCommit(t *testing.T) {
db.AddQuery(sql, sqlResult)
db.AddQuery(expanedSql, sqlResult)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -671,6 +693,7 @@ func TestSqlQueryExecuteBatchFailBeginWithoutCommit(t *testing.T) {
func TestSqlQueryExecuteBatchSqlExecFailNotInTransaction(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
sql := "insert into test_table values (1, 2)"
sqlResult := &mproto.QueryResult{}
expanedSql := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */"
@ -681,11 +704,10 @@ func TestSqlQueryExecuteBatchSqlExecFailNotInTransaction(t *testing.T) {
// cause execution error for this particular sql query
db.AddRejectedQuery(sql)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -713,10 +735,11 @@ func TestSqlQueryExecuteBatchSqlExecFailNotInTransaction(t *testing.T) {
func TestSqlQueryExecuteBatchCallCommitWithoutABegin(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -744,18 +767,17 @@ func TestSqlQueryExecuteBatchCallCommitWithoutABegin(t *testing.T) {
func TestExecuteBatchNestedTransaction(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
sql := "insert into test_table values (1, 2)"
sqlResult := &mproto.QueryResult{}
expanedSql := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */"
db.AddQuery(sql, sqlResult)
db.AddQuery(expanedSql, sqlResult)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -804,12 +826,11 @@ func TestExecuteBatchNestedTransaction(t *testing.T) {
func TestSqlQuerySplitQuery(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -843,12 +864,11 @@ func TestSqlQuerySplitQuery(t *testing.T) {
func TestSqlQuerySplitQueryInvalidQuery(t *testing.T) {
setUpSqlQueryTest()
config := newTestSqlQueryConfig()
testUtils := newTestUtils()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -882,6 +902,7 @@ func TestSqlQuerySplitQueryInvalidQuery(t *testing.T) {
func TestSqlQuerySplitQueryInvalidMinMax(t *testing.T) {
db := setUpSqlQueryTest()
testUtils := newTestUtils()
pkMinMaxQuery := "SELECT MIN(pk), MAX(pk) FROM test_table"
pkMinMaxQueryResp := &mproto.QueryResult{
Fields: []mproto.Field{
@ -898,11 +919,10 @@ func TestSqlQuerySplitQueryInvalidMinMax(t *testing.T) {
}
db.AddQuery(pkMinMaxQuery, pkMinMaxQueryResp)
config := newTestSqlQueryConfig()
config := testUtils.newQueryServiceConfig()
sqlQuery := NewSqlQuery(config)
dbconfigs := newTestDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, newMysqld(&dbconfigs))
dbconfigs := testUtils.newDBConfigs()
err := sqlQuery.allowQueries(&dbconfigs, []SchemaOverride{}, testUtils.newMysqld(&dbconfigs))
if err != nil {
t.Fatalf("allowQueries failed: %v", err)
}
@ -1017,44 +1037,6 @@ func setUpSqlQueryTest() *fakesqldb.DB {
return db
}
func newTestSqlQueryConfig() Config {
randID := rand.Int63()
config := DefaultQsConfig
config.StatsPrefix = fmt.Sprintf("Stats-%d-", randID)
config.DebugURLPrefix = fmt.Sprintf("/debug-%d-", randID)
config.RowCache.StatsPrefix = fmt.Sprintf("Stats-%d-", randID)
config.PoolNamePrefix = fmt.Sprintf("Pool-%d-", randID)
config.StrictMode = true
config.RowCache.Binary = "ls"
config.RowCache.Connections = 100
return config
}
func newMysqld(dbconfigs *dbconfigs.DBConfigs) *mysqlctl.Mysqld {
randID := rand.Int63()
return mysqlctl.NewMysqld(
fmt.Sprintf("Dba_%d", randID),
fmt.Sprintf("App_%d", randID),
mysqlctl.NewMycnf(0, 6802),
&dbconfigs.Dba,
&dbconfigs.App.ConnParams,
&dbconfigs.Repl,
)
}
func newTestDBConfigs() dbconfigs.DBConfigs {
appDBConfig := dbconfigs.DBConfig{
ConnParams: sqldb.ConnParams{},
Keyspace: "test_keyspace",
Shard: "0",
EnableRowcache: false,
EnableInvalidator: false,
}
return dbconfigs.DBConfigs{
App: appDBConfig,
}
}
func checkSqlQueryState(t *testing.T, sqlQuery *SqlQuery, expectState string) {
if sqlQuery.GetState() != expectState {
t.Fatalf("sqlquery should in state: %s, but get state: %s", expectState, sqlQuery.GetState())

Просмотреть файл

@ -270,7 +270,7 @@ func newTestTableInfo(cachePool *CachePool, tableType string, comment string) (*
dbaParams := sqldb.ConnParams{}
connPoolIdleTimeout := 10 * time.Second
connPool := NewConnPool("", 2, connPoolIdleTimeout)
connPool := NewConnPool("", 2, connPoolIdleTimeout, false)
connPool.Open(&appParams, &dbaParams)
conn, err := connPool.Get(ctx)
if err != nil {
@ -295,7 +295,7 @@ func newTestTableInfoCachePool() *CachePool {
randID := rand.Int63()
name := fmt.Sprintf("TestCachePool-TableInfo-%d-", randID)
statsURL := fmt.Sprintf("/debug/tableinfo-cache-%d", randID)
return NewCachePool(name, rowCacheConfig, 1*time.Second, statsURL)
return NewCachePool(name, rowCacheConfig, 1*time.Second, statsURL, false)
}
func getTestTableInfoQueries() map[string]*mproto.QueryResult {

Просмотреть файл

@ -111,8 +111,8 @@ func IsConnErr(err error) bool {
return false
}
}
// 2013 means that someone sniped the query.
if sqlError == 2013 {
// ErrServerLost means that someone sniped the query.
if sqlError == mysql.ErrServerLost {
return false
}
return sqlError >= 2000 && sqlError <= 2018
@ -122,6 +122,7 @@ func (te *TabletError) Error() string {
return te.Prefix() + te.Message
}
// Prefix returns the prefix for the error, like error, fatal, etc.
func (te *TabletError) Prefix() string {
prefix := "error: "
switch te.ErrorType {
@ -134,6 +135,10 @@ func (te *TabletError) Prefix() string {
case ErrNotInTx:
prefix = "not_in_tx: "
}
// Special case for killed queries.
if te.SqlError == mysql.ErrServerLost {
prefix = prefix + "the query was killed either because it timed out or was canceled: "
}
return prefix
}

Просмотреть файл

@ -46,10 +46,14 @@ func TestTabletErrorConnError(t *testing.T) {
if !IsConnErr(tabletErr) {
t.Fatalf("table error: %v is a connection error", tabletErr)
}
tabletErr = NewTabletErrorSql(ErrFatal, sqldb.NewSqlError(2013, "test"))
tabletErr = NewTabletErrorSql(ErrFatal, sqldb.NewSqlError(mysql.ErrServerLost, "test"))
if IsConnErr(tabletErr) {
t.Fatalf("table error: %v is not a connection error", tabletErr)
}
want := "fatal: the query was killed either because it timed out or was canceled: test (errno 2013)"
if tabletErr.Error() != want {
t.Fatalf("tablet error: %v, want %s", tabletErr, want)
}
sqlErr := sqldb.NewSqlError(1998, "test")
if IsConnErr(sqlErr) {
t.Fatalf("sql error: %v is not a connection error", sqlErr)

Просмотреть файл

@ -9,8 +9,13 @@ import (
"html/template"
"math/rand"
"reflect"
"strings"
"testing"
"time"
"github.com/youtube/vitess/go/sqldb"
"github.com/youtube/vitess/go/vt/dbconfigs"
"github.com/youtube/vitess/go/vt/mysqlctl"
)
type fakeCallInfo struct {
@ -38,16 +43,97 @@ func (fci *fakeCallInfo) HTML() template.HTML {
type testUtils struct{}
func newTestUtils() *testUtils {
return &testUtils{}
}
func (util *testUtils) checkEqual(t *testing.T, expected interface{}, result interface{}) {
if !reflect.DeepEqual(expected, result) {
t.Fatalf("expect to get: %v, but got: %v", expected, result)
}
}
func (util *testUtils) checkTabletErrorWithRecover(t *testing.T, tabletErrType int, tabletErrStr string) {
err := recover()
if err == nil {
t.Fatalf("should get error")
}
util.checkTabletError(t, err, tabletErrType, tabletErrStr)
}
func (util *testUtils) checkTabletError(t *testing.T, err interface{}, tabletErrType int, tabletErrStr string) {
tabletError, ok := err.(*TabletError)
if !ok {
t.Fatalf("should return a TabletError, but got err: %v", err)
}
if tabletError.ErrorType != tabletErrType {
t.Fatalf("should return a TabletError with error type: %s", util.getTabletErrorString(tabletErrType))
}
if !strings.Contains(tabletError.Error(), tabletErrStr) {
t.Fatalf("expect the tablet error should contain string: '%s', but it does not. Got tablet error: '%s'", tabletErrStr, tabletError.Error())
}
}
func (util *testUtils) getTabletErrorString(tabletErrorType int) string {
switch tabletErrorType {
case ErrFail:
return "ErrFail"
case ErrRetry:
return "ErrRetry"
case ErrFatal:
return "ErrFatal"
case ErrTxPoolFull:
return "ErrTxPoolFull"
case ErrNotInTx:
return "ErrNotInTx"
}
return ""
}
func (util *testUtils) newMysqld(dbconfigs *dbconfigs.DBConfigs) *mysqlctl.Mysqld {
randID := rand.Int63()
return mysqlctl.NewMysqld(
fmt.Sprintf("Dba_%d", randID),
fmt.Sprintf("App_%d", randID),
mysqlctl.NewMycnf(0, 6802),
&dbconfigs.Dba,
&dbconfigs.App.ConnParams,
&dbconfigs.Repl,
)
}
func (util *testUtils) newDBConfigs() dbconfigs.DBConfigs {
appDBConfig := dbconfigs.DBConfig{
ConnParams: sqldb.ConnParams{},
Keyspace: "test_keyspace",
Shard: "0",
EnableRowcache: false,
EnableInvalidator: false,
}
return dbconfigs.DBConfigs{
App: appDBConfig,
}
}
func (util *testUtils) newQueryServiceConfig() Config {
randID := rand.Int63()
config := DefaultQsConfig
config.StatsPrefix = fmt.Sprintf("Stats-%d-", randID)
config.DebugURLPrefix = fmt.Sprintf("/debug-%d-", randID)
config.RowCache.StatsPrefix = fmt.Sprintf("Stats-%d-", randID)
config.PoolNamePrefix = fmt.Sprintf("Pool-%d-", randID)
config.StrictMode = true
config.RowCache.Binary = "ls"
config.RowCache.Connections = 100
config.EnablePublishStats = false
return config
}
func newTestSchemaInfo(
queryCacheSize int,
reloadTime time.Duration,
idleTimeout time.Duration) *SchemaInfo {
idleTimeout time.Duration,
enablePublishStats bool) *SchemaInfo {
randID := rand.Int63()
return NewSchemaInfo(
queryCacheSize,
@ -59,5 +145,7 @@ func newTestSchemaInfo(
debugSchemaKey: fmt.Sprintf("/debug/schema_%d", randID),
},
reloadTime,
idleTimeout)
idleTimeout,
enablePublishStats,
)
}

Просмотреть файл

@ -65,9 +65,10 @@ func NewTxPool(
capacity int,
timeout time.Duration,
poolTimeout time.Duration,
idleTimeout time.Duration) *TxPool {
idleTimeout time.Duration,
enablePublishStats bool) *TxPool {
axp := &TxPool{
pool: NewConnPool(name, capacity, idleTimeout),
pool: NewConnPool(name, capacity, idleTimeout, enablePublishStats),
activePool: pools.NewNumbered(),
lastID: sync2.AtomicInt64(time.Now().UnixNano()),
timeout: sync2.AtomicDuration(timeout),
@ -77,8 +78,10 @@ func NewTxPool(
}
// Careful: pool also exports name+"xxx" vars,
// but we know it doesn't export Timeout.
stats.Publish(name+"Timeout", stats.DurationFunc(axp.timeout.Get))
stats.Publish(name+"PoolTimeout", stats.DurationFunc(axp.poolTimeout.Get))
if enablePublishStats {
stats.Publish(name+"Timeout", stats.DurationFunc(axp.timeout.Get))
stats.Publish(name+"PoolTimeout", stats.DurationFunc(axp.poolTimeout.Get))
}
return axp
}

Просмотреть файл

@ -21,7 +21,7 @@ func TestExecuteCommit(t *testing.T) {
tableName := "test_table"
sql := fmt.Sprintf("ALTER TABLE %s ADD test_column INT", tableName)
fakesqldb.Register()
txPool := newTxPool()
txPool := newTxPool(true)
txPool.SetTimeout(1 * time.Second)
txPool.SetPoolTimeout(1 * time.Second)
appParams := sqldb.ConnParams{}
@ -51,7 +51,7 @@ func TestExecuteCommit(t *testing.T) {
func TestExecuteRollback(t *testing.T) {
sql := "ALTER TABLE test_table ADD test_column INT"
fakesqldb.Register()
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -71,7 +71,7 @@ func TestExecuteRollback(t *testing.T) {
func TestTransactionKiller(t *testing.T) {
sql := "ALTER TABLE test_table ADD test_column INT"
fakesqldb.Register()
txPool := newTxPool()
txPool := newTxPool(false)
// make sure transaction killer will run frequent enough
txPool.SetTimeout(time.Duration(10))
appParams := sqldb.ConnParams{}
@ -94,7 +94,7 @@ func TestTransactionKiller(t *testing.T) {
func TestBeginAfterConnPoolClosed(t *testing.T) {
fakesqldb.Register()
txPool := newTxPool()
txPool := newTxPool(false)
txPool.SetTimeout(time.Duration(10))
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
@ -116,7 +116,7 @@ func TestBeginAfterConnPoolClosed(t *testing.T) {
func TestBeginWithPoolTimeout(t *testing.T) {
fakesqldb.Register()
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -135,7 +135,7 @@ func TestBeginWithPoolTimeout(t *testing.T) {
func TestBeginWithPoolConnectionError(t *testing.T) {
db := fakesqldb.Register()
db.EnableConnFail()
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -148,7 +148,7 @@ func TestBeginWithPoolConnectionError(t *testing.T) {
func TestBeginWithExecError(t *testing.T) {
db := fakesqldb.Register()
db.AddRejectedQuery("begin")
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -164,7 +164,7 @@ func TestTxPoolSafeCommitFail(t *testing.T) {
db.AddQuery("begin", &proto.QueryResult{})
db.AddQuery(sql, &proto.QueryResult{})
db.AddRejectedQuery("commit")
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -187,7 +187,7 @@ func TestTxPoolRollbackFail(t *testing.T) {
db := fakesqldb.Register()
db.AddRejectedQuery("rollback")
sql := "ALTER TABLE test_table ADD test_column INT"
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -207,7 +207,7 @@ func TestTxPoolRollbackFail(t *testing.T) {
func TestTxPoolGetConnFail(t *testing.T) {
fakesqldb.Register()
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -218,7 +218,7 @@ func TestTxPoolGetConnFail(t *testing.T) {
func TestTxPoolExecFailDueToConnFail(t *testing.T) {
db := fakesqldb.Register()
txPool := newTxPool()
txPool := newTxPool(false)
appParams := sqldb.ConnParams{}
dbaParams := sqldb.ConnParams{}
txPool.Open(&appParams, &dbaParams)
@ -237,7 +237,7 @@ func TestTxPoolExecFailDueToConnFail(t *testing.T) {
}
}
func newTxPool() *TxPool {
func newTxPool(enablePublishStats bool) *TxPool {
randID := rand.Int63()
poolName := fmt.Sprintf("TestTransactionPool-%d", randID)
txStatsPrefix := fmt.Sprintf("TxStats-%d-", randID)
@ -252,6 +252,7 @@ func newTxPool() *TxPool {
transactionTimeout,
txPoolTimeout,
idleTimeout,
enablePublishStats,
)
}

Просмотреть файл

@ -9,7 +9,6 @@ import (
"html/template"
"io"
"net/http"
"strconv"
"time"
log "github.com/golang/glog"
@ -51,31 +50,6 @@ func init() {
http.HandleFunc("/txlogz", txlogzHandler)
}
func adjustValue(val int, lower int, upper int) int {
if val < lower {
return lower
} else if val > upper {
return upper
}
return val
}
func parseReqParam(req *http.Request) (time.Duration, int) {
timeout := 10
limit := 300
if ts, ok := req.URL.Query()["timeout"]; ok {
if t, err := strconv.Atoi(ts[0]); err == nil {
timeout = adjustValue(t, 0, 60)
}
}
if l, ok := req.URL.Query()["limit"]; ok {
if lim, err := strconv.Atoi(l[0]); err == nil {
limit = adjustValue(lim, 1, 200000)
}
}
return time.Duration(timeout) * time.Second, limit
}
// txlogzHandler serves a human readable snapshot of the
// current transaction log.
// Endpoint: /txlogz?timeout=%d&limit=%d
@ -87,7 +61,7 @@ func txlogzHandler(w http.ResponseWriter, req *http.Request) {
return
}
timeout, limit := parseReqParam(req)
timeout, limit := parseTimeoutLimitParams(req)
ch := TxLogger.Subscribe("txlogz")
defer TxLogger.Unsubscribe(ch)
startHTMLTable(w)

Просмотреть файл

@ -4,7 +4,11 @@ import (
"github.com/youtube/vitess/go/vt/topo"
)
// TabletChange is an event that describes changes to a tablet.
// TabletChange is an event that describes changes to a tablet's topo record.
// It is triggered when the CURRENT process changes ANY tablet's record.
// It is NOT triggered when a DIFFERENT process changes THIS tablet's record.
// To be notified when THIS tablet's record changes, even if it was changed
// by a different process, listen for go/vt/tabletmanager/events.StateChange.
type TabletChange struct {
Tablet topo.Tablet
Status string

172
test.go Normal file
Просмотреть файл

@ -0,0 +1,172 @@
// Copyright 2015, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
test.go is a "Go script" for running Vitess tests. It runs each test in its own
Docker container for hermeticity and (potentially) parallelism. If a test fails,
this script will save the output in _test/ and continue with other tests.
Before using it, you should have Docker 1.5+ installed, and have your user in
the group that lets you run the docker command without sudo. The first time you
run against a given flavor, it may take some time for the corresponding
bootstrap image (vitess/bootstrap:<flavor>) to be downloaded.
It is meant to be run from the Vitess root, like so:
~/src/github.com/youtube/vitess$ go run test.go [args]
For a list of options, run:
$ go run test.go --help
*/
package main
import (
"encoding/json"
"flag"
"fmt"
"io/ioutil"
"log"
"os"
"os/exec"
"os/signal"
"path"
"strings"
"syscall"
"time"
)
var (
flavor = flag.String("flavor", "mariadb", "bootstrap flavor to run against")
retryMax = flag.Int("retry", 3, "max number of retries, to detect flaky tests")
logPass = flag.Bool("log-pass", false, "log test output even if it passes")
timeout = flag.Duration("timeout", 10*time.Minute, "timeout for each test")
)
// Config is the overall object serialized in test/config.json.
type Config struct {
Tests []Test
}
// Test is an entry from the test/config.json file.
type Test struct {
Name, File, Args string
}
// run executes a single try.
func (t Test) run() error {
testCmd := fmt.Sprintf("make build && test/%s %s", t.File, t.Args)
dockerCmd := exec.Command("docker/test/run.sh", *flavor, testCmd)
// Kill child process if we get a signal.
sigchan := make(chan os.Signal)
signal.Notify(sigchan, syscall.SIGINT, syscall.SIGTERM)
go func() {
if _, ok := <-sigchan; ok {
if dockerCmd.Process != nil {
dockerCmd.Process.Signal(syscall.SIGTERM)
}
log.Fatalf("received signal, quitting")
}
}()
// Stop the test if it takes too long.
done := make(chan struct{})
timer := time.NewTimer(*timeout)
defer timer.Stop()
go func() {
select {
case <-done:
case <-timer.C:
t.logf("timeout exceeded")
if dockerCmd.Process != nil {
dockerCmd.Process.Signal(syscall.SIGTERM)
}
}
}()
// Run the test.
output, err := dockerCmd.CombinedOutput()
close(done)
signal.Stop(sigchan)
close(sigchan)
// Save test output.
if err != nil || *logPass {
outDir := path.Join("_test", *flavor)
outFile := path.Join(outDir, t.Name+".log")
t.logf("saving test output to %v", outFile)
if dirErr := os.MkdirAll(outDir, os.FileMode(0755)); dirErr != nil {
t.logf("Mkdir error: %v", dirErr)
}
if fileErr := ioutil.WriteFile(outFile, output, os.FileMode(0644)); fileErr != nil {
t.logf("WriteFile error: %v", fileErr)
}
}
return err
}
func (t Test) logf(format string, v ...interface{}) {
log.Printf("%v: %v", t.Name, fmt.Sprintf(format, v...))
}
func main() {
flag.Parse()
// Get test configs.
configData, err := ioutil.ReadFile("test/config.json")
if err != nil {
log.Fatalf("Can't read config file: %v", err)
}
var config Config
if err := json.Unmarshal(configData, &config); err != nil {
log.Fatalf("Can't parse config file: %v", err)
}
log.Printf("Bootstrap flavor: %v", *flavor)
// Keep stats.
failed := 0
passed := 0
flaky := 0
// Run tests.
for _, test := range config.Tests {
if test.Name == "" {
test.Name = strings.TrimSuffix(test.File, ".py")
}
for try := 1; ; try++ {
if try > *retryMax {
// Every try failed.
test.logf("retry limit exceeded")
failed++
break
}
test.logf("running (try %v/%v)...", try, *retryMax)
start := time.Now()
if err := test.run(); err != nil {
// This try failed.
test.logf("FAILED (try %v/%v): %v", try, *retryMax, err)
continue
}
if try == 1 {
// Passed on the first try.
test.logf("PASSED in %v", time.Since(start))
passed++
} else {
// Passed, but not on the first try.
test.logf("FLAKY (1/%v passed)", try)
flaky++
}
break
}
}
// Print stats.
log.Printf("%v PASSED, %v FLAKY, %v FAILED", passed, flaky, failed)
if failed > 0 {
os.Exit(1)
}
}

80
test/config.json Normal file
Просмотреть файл

@ -0,0 +1,80 @@
{
"Tests": [
{
"Name": "queryservice_vtocc",
"File": "queryservice_test.py",
"Args": "-m -e vtocc"
},
{
"Name": "queryservice_vttablet",
"File": "queryservice_test.py",
"Args": "-m -e vttablet"
},
{
"File": "vertical_split.py"
},
{
"File": "vertical_split_vtgate.py"
},
{
"File": "schema.py"
},
{
"File": "keyspace_test.py"
},
{
"File": "keyrange_test.py"
},
{
"File": "mysqlctl.py"
},
{
"File": "sharded.py"
},
{
"File": "secure.py"
},
{
"File": "binlog.py"
},
{
"File": "clone.py"
},
{
"File": "update_stream.py"
},
{
"File": "tabletmanager.py"
},
{
"File": "reparent.py"
},
{
"File": "vtdb_test.py"
},
{
"File": "vtgate_utils_test.py"
},
{
"File": "rowcache_invalidator.py"
},
{
"File": "vtgatev2_test.py"
},
{
"File": "zkocc_test.py"
},
{
"File": "initial_sharding_bytes.py"
},
{
"File": "initial_sharding.py"
},
{
"File": "resharding_bytes.py"
},
{
"File": "resharding.py"
}
]
}

Просмотреть файл

@ -119,6 +119,13 @@ class MariaDB(MysqlFlavor):
def disable_binlog_checksum(self, tablet):
tablet.mquery('', 'SET @@global.binlog_checksum=0')
class MySQL56(MysqlFlavor):
"""Overrides specific to MySQL 5.6"""
def bootstrap_archive(self):
return "mysql-db-dir_5.6.24.tbz"
__mysql_flavor = None
@ -146,9 +153,8 @@ def set_mysql_flavor(flavor):
if flavor == "MariaDB":
__mysql_flavor = MariaDB()
elif flavor == "Mysql56":
logging.error("Mysql56 support is currently under development, and not supported yet")
exit(1)
elif flavor == "MySQL56":
__mysql_flavor = MySQL56()
else:
logging.error("Unknown MYSQL_FLAVOR '%s'", flavor)
exit(1)

Просмотреть файл

@ -367,7 +367,7 @@ class TestNocache(framework.TestCase):
conn.begin()
cu.execute("select sleep(0.5) from vtocc_test", {})
except dbexceptions.DatabaseError as e:
if "error: Query" not in str(e) and "error: Lost connection" not in str(e):
if "error: Query" not in str(e) and "error: the query was killed" not in str(e):
self.fail("Query not killed as expected")
else:
self.fail("Did not receive exception")

Просмотреть файл

@ -112,7 +112,7 @@ class TestStream(framework.TestCase):
thd.join()
with self.assertRaises(dbexceptions.DatabaseError) as cm:
cu.fetchall()
errMsg1 = "error: Lost connection to MySQL server during query (errno 2013)"
errMsg1 = "error: the query was killed either because it timed out or was canceled: Lost connectioy to MySQL server during query (errno 2013)"
errMsg2 = "error: Query execution was interrupted (errno 1317)"
self.assertTrue(cm.exception not in (errMsg1, errMsg2), "did not raise interruption error: %s" % str(cm.exception))
cu.close()