Create & Init Project...

This commit is contained in:
2019-04-22 18:49:16 +08:00
commit fc4fa37393
25440 changed files with 4054998 additions and 0 deletions

View File

@@ -0,0 +1,37 @@
package(default_visibility = ["//visibility:public"])
load(
"@io_bazel_rules_go//go:def.bzl",
"go_library",
)
go_library(
name = "go_default_library",
srcs = [
"offset.go",
"reader.go",
],
importpath = "go-common/app/infra/canal/service/reader",
tags = ["automanaged"],
visibility = ["//visibility:public"],
deps = [
"//library/log:go_default_library",
"//vendor/github.com/Shopify/sarama:go_default_library",
"//vendor/github.com/pingcap/tidb-tools/tidb_binlog/slave_binlog_proto/go-binlog:go_default_library",
"//vendor/github.com/pkg/errors:go_default_library",
],
)
filegroup(
name = "package-srcs",
srcs = glob(["**"]),
tags = ["automanaged"],
visibility = ["//visibility:private"],
)
filegroup(
name = "all-srcs",
srcs = [":package-srcs"],
tags = ["automanaged"],
visibility = ["//visibility:public"],
)

View File

@@ -0,0 +1,173 @@
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package reader
import (
"go-common/library/log"
"github.com/Shopify/sarama"
pb "github.com/pingcap/tidb-tools/tidb_binlog/slave_binlog_proto/go-binlog"
pkgerr "github.com/pkg/errors"
)
// KafkaSeeker seeks offset in kafka topics by given condition
type KafkaSeeker struct {
consumer sarama.Consumer
client sarama.Client
}
// NewKafkaSeeker creates an instance of KafkaSeeker
func NewKafkaSeeker(addr []string, config *sarama.Config) (*KafkaSeeker, error) {
client, err := sarama.NewClient(addr, config)
if err != nil {
return nil, pkgerr.WithStack(err)
}
consumer, err := sarama.NewConsumerFromClient(client)
if err != nil {
return nil, pkgerr.WithStack(err)
}
s := &KafkaSeeker{
client: client,
consumer: consumer,
}
return s, nil
}
// Close releases resources of KafkaSeeker
func (ks *KafkaSeeker) Close() {
ks.consumer.Close()
ks.client.Close()
}
// Seek seeks the first offset which binlog CommitTs bigger than ts
func (ks *KafkaSeeker) Seek(topic string, ts int64, partitions []int32) (offsets []int64, err error) {
if len(partitions) == 0 {
partitions, err = ks.consumer.Partitions(topic)
if err != nil {
log.Error("tidb get partitions from topic %s error %v", topic, err)
return nil, pkgerr.WithStack(err)
}
}
offsets, err = ks.seekOffsets(topic, partitions, ts)
if err != nil {
err = pkgerr.WithStack(err)
log.Error("tidb seek offsets error %v", err)
}
return
}
func (ks *KafkaSeeker) getTSFromMSG(msg *sarama.ConsumerMessage) (ts int64, err error) {
binlog := new(pb.Binlog)
err = binlog.Unmarshal(msg.Value)
if err != nil {
err = pkgerr.WithStack(err)
return
}
return binlog.CommitTs, nil
}
// seekOffsets returns all valid offsets in partitions
func (ks *KafkaSeeker) seekOffsets(topic string, partitions []int32, pos int64) ([]int64, error) {
offsets := make([]int64, len(partitions))
for _, partition := range partitions {
start, err := ks.client.GetOffset(topic, partition, sarama.OffsetOldest)
if err != nil {
err = pkgerr.WithStack(err)
return nil, err
}
end, err := ks.client.GetOffset(topic, partition, sarama.OffsetNewest)
if err != nil {
err = pkgerr.WithStack(err)
return nil, err
}
offset, err := ks.seekOffset(topic, partition, start, end-1, pos)
if err != nil {
err = pkgerr.WithStack(err)
return nil, err
}
offsets[partition] = offset
}
return offsets, nil
}
func (ks *KafkaSeeker) seekOffset(topic string, partition int32, start int64, end int64, ts int64) (offset int64, err error) {
startTS, err := ks.getTSAtOffset(topic, partition, start)
if err != nil {
err = pkgerr.WithStack(err)
return
}
if ts < startTS {
log.Warn("given ts %v is smaller than oldest message's ts %v, some binlogs may lose", ts, startTS)
offset = start
return
} else if ts == startTS {
offset = start + 1
return
}
for start < end {
mid := (end-start)/2 + start
var midTS int64
midTS, err = ks.getTSAtOffset(topic, partition, mid)
if err != nil {
err = pkgerr.WithStack(err)
return
}
if midTS <= ts {
start = mid + 1
} else {
end = mid
}
}
var endTS int64
endTS, err = ks.getTSAtOffset(topic, partition, end)
if err != nil {
err = pkgerr.WithStack(err)
return
}
if endTS <= ts {
return sarama.OffsetNewest, nil
}
return end, nil
}
func (ks *KafkaSeeker) getTSAtOffset(topic string, partition int32, offset int64) (ts int64, err error) {
pc, err := ks.consumer.ConsumePartition(topic, partition, offset)
if err != nil {
err = pkgerr.WithStack(err)
return
}
defer pc.Close()
for msg := range pc.Messages() {
ts, err = ks.getTSFromMSG(msg)
err = pkgerr.WithStack(err)
return
}
panic("unreachable")
}

View File

@@ -0,0 +1,174 @@
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package reader
import (
"fmt"
"go-common/library/log"
"github.com/Shopify/sarama"
pb "github.com/pingcap/tidb-tools/tidb_binlog/slave_binlog_proto/go-binlog"
pkgerr "github.com/pkg/errors"
)
func init() {
// log.SetLevel(log.LOG_LEVEL_NONE)
sarama.MaxResponseSize = 1 << 30
}
// Config for Reader
type Config struct {
KafkaAddr []string
// the CommitTs of binlog return by reader will bigger than the config CommitTs
CommitTS int64
Offset int64 // start at kafka offset
ClusterID string
Name string
}
// Message read from reader
type Message struct {
Binlog *pb.Binlog
Offset int64 // kafka offset
}
// Reader to read binlog from kafka
type Reader struct {
cfg *Config
client sarama.Client
msgs chan *Message
stop chan struct{}
clusterID string
}
func (r *Reader) getTopic() (string, int32) {
return r.cfg.ClusterID + "_obinlog", 0
}
func (r *Reader) name() string {
return fmt.Sprintf("%s-%s", r.cfg.Name, r.cfg.ClusterID)
}
// NewReader creates an instance of Reader
func NewReader(cfg *Config) (r *Reader, err error) {
r = &Reader{
cfg: cfg,
stop: make(chan struct{}),
msgs: make(chan *Message, 1024),
clusterID: cfg.ClusterID,
}
r.client, err = sarama.NewClient(r.cfg.KafkaAddr, nil)
if err != nil {
err = pkgerr.WithStack(err)
r = nil
return
}
if (r.cfg.Offset == 0) && (r.cfg.CommitTS > 0) {
r.cfg.Offset, err = r.getOffsetByTS(r.cfg.CommitTS)
if err != nil {
err = pkgerr.WithStack(err)
r = nil
return
}
log.Info("tidb %s: set offset to: %v", r.name(), r.cfg.Offset)
}
return
}
// Close shuts down the reader
func (r *Reader) Close() {
close(r.stop)
r.client.Close()
}
// Messages returns a chan that contains unread buffered message
func (r *Reader) Messages() (msgs <-chan *Message) {
return r.msgs
}
func (r *Reader) getOffsetByTS(ts int64) (offset int64, err error) {
seeker, err := NewKafkaSeeker(r.cfg.KafkaAddr, nil)
if err != nil {
err = pkgerr.WithStack(err)
return
}
topic, partition := r.getTopic()
offsets, err := seeker.Seek(topic, ts, []int32{partition})
if err != nil {
err = pkgerr.WithStack(err)
return
}
offset = offsets[0]
return
}
// Run start consume msg
func (r *Reader) Run() {
offset := r.cfg.Offset
log.Info("tidb %s start at offset: %v", r.name(), offset)
consumer, err := sarama.NewConsumerFromClient(r.client)
if err != nil {
log.Error("tidb %s NewConsumerFromClient err: %v", r.name(), err)
return
}
defer consumer.Close()
topic, partition := r.getTopic()
partitionConsumer, err := consumer.ConsumePartition(topic, partition, offset)
if err != nil {
log.Error("tidb %s ConsumePartition err: %v", r.name(), err)
return
}
defer partitionConsumer.Close()
for {
select {
case <-r.stop:
partitionConsumer.Close()
close(r.msgs)
log.Info("tidb %s reader stop to run", r.name())
return
case kmsg, ok := <-partitionConsumer.Messages():
if !ok {
close(r.msgs)
log.Info("tidb %s reader stop to run because partitionConsumer close", r.name())
return
}
if kmsg == nil {
continue
}
log.Info("tidb %s get kmsg offset: %v", r.name(), kmsg.Offset)
binlog := new(pb.Binlog)
err := binlog.Unmarshal(kmsg.Value)
if err != nil {
log.Warn("%s unmarshal err %+v", r.name(), err)
continue
}
if r.cfg.CommitTS > 0 && binlog.CommitTs <= r.cfg.CommitTS {
log.Warn("%s skip binlog CommitTs: ", r.name(), binlog.CommitTs)
continue
}
r.msgs <- &Message{
Binlog: binlog,
Offset: kmsg.Offset,
}
}
}
}