diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000..9b5001b --- /dev/null +++ b/.gitignore @@ -0,0 +1,6 @@ +.history/ +.DS_Store +cmd/influxd-ctl/influxd-ctl +cmd/influxd/influxd +cmd/metad/metad +sync_simulation diff --git a/Data_Cluster_Maintenance.md b/Data_Cluster_Maintenance.md new file mode 100644 index 0000000..af33476 --- /dev/null +++ b/Data_Cluster_Maintenance.md @@ -0,0 +1,94 @@ +# Data Cluster Maintenance + +## Get Status of Cluster + +### Node List + +Use following to list all data nodes in cluster (no matter alive or dead): + +```shell +influxd-ctl -s ip:port node list +``` + +Where `ip:port` is any **TCP address** of **alive** node in this cluster. + +Sample output as: + +```shell +Nodes: +4 http://:8092 tcp://127.0.0.1:8082 +5 http://:8093 tcp://127.0.0.1:8083 +6 http://:8094 tcp://127.0.0.1:8084 +7 http://:8095 tcp://127.0.0.1:8085 +8 http://:8096 tcp://127.0.0.1:8086 +9 http://:8091 tcp://127.0.0.1:8081 +15 http://:8097 tcp://127.0.0.1:8087 +``` + +### Shards on Node + +Use following to list all available shards (only id) on specific node: + +```shell +influxd-ctl -s ip:addr shard node +``` + +Output: + +```shell +Shards on node 15: +[513 549 556 575 578 580 582 585 593 594] +``` + +### Shards of Retention Policy + +```shell +influxd-ctl -s ip:port shard list +``` + +### Single Shard Info + +```shell +influxd-ctl -s ip:port shard info +``` + +Output: + +```shell +Shard: 594 +Database: _internal +Retention Policy: monitor +Nodes: [15] +``` + +## Restart Node + +Feel free to restart any node if you have **handoff hinted**(hh) service enabled +on every other node. New replicated data blocked will be cached and retry to +replicate when the node was back online. Any failed query sent to this node will +be retried to other replicas. + +## Add New Node + +Adding operation is simple. Configure it and start it then it will appear in +node list. + +## Remove Node + +1. Remove it from configuration through `influxd-ctl node remove` +2. Stop the instance + +## Replace Node + +Replacement is more complicated. For instance we call the instance to be replaced +as `A` and the new one as `B`. + +1. Add B into cluster +2. Freeze both A and B through `influxd-ctl node freeze` +3. Truncate shards and wait a while to make sure no further writes on A and B +4. Get all shards through `influxd-ctl shard node` +5. Copy them from A to B through `influxd-ctl shard copy` +6. Progress can be checked through `influxd-ctl shard status` +7. Better to verify the actual data directories are copied correctly +8. Remove A from cluster +9. Unfreeze B to let it accept creation of new shards diff --git a/Meta_Cluster_Maintenance.md b/Meta_Cluster_Maintenance.md new file mode 100644 index 0000000..06a6a5e --- /dev/null +++ b/Meta_Cluster_Maintenance.md @@ -0,0 +1,79 @@ +# Meta Cluster Maintenance + +## Get Status of Cluster + +```shell +metad-ctl status -s ip:port +``` + +Sample output as: + +```shell +Cluster: +Leader: 3 +Term: 8 +Committed: 4685619 +Applied: 4685619 + +Nodes: +1 Follower 127.0.0.1:2345 StateReplicate 4685619=>4685620 +2 Follower 127.0.0.1:2346 StateReplicate 4685619=>4685620 Vote(3) +3 Leader 127.0.0.1:2347 StateReplicate 4685619=>4685620 Vote(3) +``` + +## Restart Node + +### Restart Follower + +Feel free to restart any follower. The only thing you should take care is that +it's better to restart one node at a time and make sure the status of cluster +become healthy again. + +### Restart Leader + +Restart a leader should follow 2 phases: + +1. Kill the leader +2. Check status of cluster whether a new leader has been elected +3. Start it and now it's a follower + +## Add New Node + +Adding operation should also follow 2 phases step. + +1. Add it into the configuration using `metad-ctl add` specifying `id` and `addr` +2. Start the new, empty meta node + +One node at a time. If you want to add multiple nodes just repeat the below steps. + +## Remove Node + +1. Kill it +2. Remove it from configuration using `metad-ctl remove` + +## Replace Node + +There are two strategies to replace an existing node. + +First is remove-add strategy. +In this strategy you can remove it first follow steps in `Remove Node` and then +add a new node follow steps in `Add New Node`. The core point is that you can +use the same **address** / **id** of the removed one. + +Another is add-remove strategy. +In this strategy you first add a new node into cluster and then remove the old +one. The core point is that it maybe safer compared to first strategy. But you +can't use the same id or address because they will both up for a while. + +## Recover from Disaster + +If something bad happened and the cluster wouldn't achieve consensus anymore or +there was other reason which caused cluster can't work anymore, here is how to get +them back. + +First you should check which storage of node you want to recover with. Use commend +`metad -config -dump a.db` to dump the storage to the file `a.db`. + +Second you can boot up the first node using it through `metad -config -restore a.db`. +Now you have a single-instance cluster. Then you can follow the `Add New Node` steps +to add the rest one by one. diff --git a/README.md b/README.md index 896f287..b19cdf9 100644 --- a/README.md +++ b/README.md @@ -1,78 +1,190 @@ -# chronus -chronus是基于influxdb1.7.x版本开发的分布式时序数据库,兼容官方的influxql。 +# Chronus + +Chronus is an opensourced version of implementation follow the +design of official influxdb-cluster. +It's free and compatible with original influxdb. ## Features -- 完全兼容influxdb的influxql协议 +- Compatible with influxql +- High availability +- Scalable online +- Eventual consistent +- Parallel computation +- No other infrastructure needed + +## Architecture -- 高可用 +![Cluster Sample](docs/architecture.png) -- 水平线性扩展 +### Meta Server -- 最终一致性 +Keep the meta data including shards and its group, continuous +queries, data nodes, etc. . -- MPP架构:大规模并行处理 +High availability of meta cluster is achieved through RAFT +(from etcd). So at least 3 meta servers are recommended. And +any odd number greater than one should work but the more the +poorer writing performance. So 3 or 5 is recommended finally. -- 不依赖第三方服务 +Meta server cluster can be managed through command line tool +`metad-ctl` in `cmd/metad-ctl`. -## Architecture - +### Data Server + +Data servers hold the data of shards, run continuous queries +(when aquired lock), accept queries of client and do inner +remote joins if necessary. + +Shards are managed by shard groups in logical and each shard +will be guaranteed to be owned by specified `replica` count of +data servers. Group is used to make writes distributed. + +## Compile + +```shell +cd cmd/influxd && go build && cd - +cd cmd/influxd-ctl && go build && cd - +cd cmd/metad && go build && cd - +cd cmd/metad-ctl && go build && cd - +``` + +You would get four binaries (metad, metad-ctl, influxd, +influxd-ctl) if successfully. You can put them into PATH or +package them into private repository. + +TIPS for proxy: + +```shell +export GO111MODULE=on +export GOPROXY=https://goproxy.io +``` + +## Boot Meta Cluster + +### Configuration +Sample configuration can be reached by executing `metad -sample`. +You should pay attention to following items: -- meta server:负责元数据的存储和管理 +- my-addr: in ip:port format which will be used in cluster's communication between nodes +- raft-id: should be a positive, unique number +- wal-dir: where the meta data stores in +- log-dir: where the logs store in. Logs will be splitted automatically and if you leave +it blank, standard output would be used +- log-level: in production `warn` is recommended -- data server:负责数据的存储和管理,并向meta server获取元数据。所有涉及到元数据的修改,都会请求到meta server,由meta server执行修改,并同步给集群内所有的data server +### Boot First Meta Node -## compile +Start with sample configuation: -- export GO111MODULE=on +```shell +metad -sample > metad.conf +metad -config metad.conf +``` -- 需要梯子:export -GOPROXY=https://goproxy.io +Then a single node meta cluster is done. -- mkdir -p workspace/src/github.com/angopher && export GOPATH=$PWD/workspace +### Check Meta Cluster Status -- cd workspace/src/github.com/angopher +Using: -- git clone https://github.com/angopher/chronus.git && cd chronus +```shell +metad-ctl status -s ip:port +``` -- 编译influxd: cd cmd/influxd && go build && cd - +You can specify any **alive** address in cluster by `-s ip:port` and it's +always required to call `metad-ctl`. -- 编译metad:cd cmd/metad && go build +### Add More Meta Nodes to Existed Cluster -## run +You can add more nodes into cluster through two phases: -配置启动meta server +1. Add node through `metad-ctl add` before boot up new node actually +2. Boot up the node with correct configuration, thus, correct `raft-id` / `my-addr` +3. [CHECK]Always check the cluster status and make sure everything as your expected. -- 生成默认配置文件:./metad config > metad.conf +Please add ONLY one node at a time. -- ./metad -config metad.conf +> You can also boot up the cluster initially with configured `peers` altogether. +> Set peers expected, boot them up. -配置启动data server +```text +Sample peers format: +peers=[ + {addr="127.0.0.1:2345", raft-id=1}, + {addr="127.0.0.1:2346", raft-id=2}, + {addr="127.0.0.1:2347", raft-id=3}, +] +``` -- 生成默认配置文件 ./influxd config > influxd.conf +## Boot Data Cluster -- ./influxd -config influxd.conf +You can use following commands to generate sample configuration of data node. -## Getting Started +```shell +influxd config > data.conf +``` -创建第一个数据库 -- curl -XPOST "http://localhost:8086/query" --data-urlencode "q=CREATE DATABASE mydb" +You may mainly care about following configurations: -写入数据 -- curl -XPOST "http://localhost:8086/write?db=mydb" -d 'cpu,host=server01,region=uswest load=42 1434055562000000000' -- curl -XPOST "http://localhost:8086/write?db=mydb" -d 'cpu,host=server02,region=uswest load=78 1434055562000000000' -- curl -XPOST "http://localhost:8086/write?db=mydb" -d 'cpu,host=server03,region=useast load=15.4 1434055562000000000' +- bind-address: TCP address of data node. +- meta.dir: Directory holds meta data. Absolute path is recommended. +- data.dir: Directory holds data. Absolute path is recommended. +- data.wal-dir: Directory holds wal data. Absolute path is recommended. +- coordinator.pool-max-streams-per-node: Max streams allowed to single data node when +forwarding quries internally. You can adjust it according to your load. +- coordinator.meta-services: [Important]Addresses of meta nodes. +- http.bind-address: Query service listening address which is also called `HTTP Address`. +- http.access-log-path: File holds access log. It will be rotated automatically. Leave it +empty to disable. +- logging.format: `console`(`auto`) / `json` / `logfmt` +- logging.level: `debug` / `info` / `warn` / `fatal` +- continuous_queries.run-interval: Interval of running continuous queries. +- hinted-handoff.{enabled, dir}: If you want to use hh service you should set both of them. +- hinted-handoff.retry-concurrency: If you want to constraint the max requests retrying. Or +unlimited retrying may exhaust the connection pool quickly. +- controller.max_shard_copy_tasks: Max concurrency of active copying task on node. -查询 -- curl -G "http://localhost:8086/query?pretty=true" --data-urlencode "db=mydb" --data-urlencode "q=SELECT * FROM cpu WHERE host='server01' AND time < now() - 1d" +You can start the data node using: -分析 -- curl -G "http://localhost:8086/query?pretty=true" --data-urlencode "db=mydb" --data-urlencode "q=SELECT mean(load) FROM cpu WHERE region='uswest'" +```shell +influxd run -config data.conf +``` + +or you want all the logs write into a directory instead of standard output: + +```shell +influxd run -config data.conf -logdir /dir/to/write +``` + +You can repeat to add more data nodes and check the node list using: + +```shell +influxd-ctl -s ip:port node list +``` + +Where `ip:port` is the TCP address of any node in data node cluster. + +## Query + +Data cluster is compatible with `influx` command line tool and any other clients. +Pay attention to the following rules: + +- All data nodes can be quried equivalently. +- Append only. +- Carefully set retention policies. + +## Maintenance + +Maintain meta cluster please check [Meta Cluster Maintenance](Meta_Cluster_Maintenance.md) + +Maintain data cluster please check [Data Cluster Maintenance](Data_Cluster_Maintenance.md) ## License -chronus is under the MIT license. See the [LICENSE](LICENSE) file for details. -欢迎微信交流 +Chronus is under the MIT license. See the [LICENSE](LICENSE) file for details. + +## Contact - +![WeChat QR Code](./docs/wechat.jpeg) diff --git a/cmd/influxd-ctl/action/action.go b/cmd/influxd-ctl/action/action.go index f344122..7633429 100644 --- a/cmd/influxd-ctl/action/action.go +++ b/cmd/influxd-ctl/action/action.go @@ -2,14 +2,17 @@ package action import ( "encoding/json" + "errors" "fmt" - "io" "net" + "sort" "strconv" "time" + "github.com/angopher/chronus/cmd/metad-ctl/util" "github.com/angopher/chronus/coordinator" "github.com/angopher/chronus/services/controller" + "github.com/fatih/color" ) func CopyShard(srcAddr, dstAddr, shardID string) error { @@ -54,6 +57,133 @@ func TruncateShards(delay string, addr string) error { return nil } +func ListShardOnNode(addr string, nodeId uint64) error { + var req controller.GetNodeShardsRequest + var resp controller.NodeShardsResponse + respTyp := byte(controller.ResponseNodeShards) + reqTyp := byte(controller.RequestNodeShards) + req.NodeID = nodeId + var err error + if err = RequestAndWaitResp(addr, reqTyp, respTyp, req, &resp); err != nil { + return err + } + if resp.Code != 0 { + return errors.New(resp.Msg) + } + + color.Set(color.Bold) + color.Green(fmt.Sprint("Shards on node ", req.NodeID, ":\n")) + fmt.Println(resp.Shards) + fmt.Println() + return nil +} + +func ListShard(addr, db, rp string) error { + var req controller.GetShardsRequest + var resp controller.ShardsResponse + respTyp := byte(controller.ResponseShards) + reqTyp := byte(controller.RequestShards) + req.Database = db + req.RetentionPolicy = rp + nodes, err := getNodes(addr) + if err != nil { + return err + } + if err = RequestAndWaitResp(addr, reqTyp, respTyp, req, &resp); err != nil { + return err + } + if resp.Code != 0 { + return errors.New(resp.Msg) + } + + if resp.Rp == "" { + return errors.New("Specified retention policy could not be found") + } + + color.Set(color.Bold) + color.Green("Retenion Policy [%s]:\n", resp.Rp) + fmt.Println("Replica:", resp.Replica) + fmt.Println("Duration:", resp.Duration) + fmt.Println("Group Duration:", resp.GroupDuration) + fmt.Println() + + color.Set(color.Bold) + color.Green("Groups\n") + color.Yellow(fmt.Sprint( + util.PadRight("GroupId", 10), + util.PadRight("Start", 21), + util.PadRight("End", 21), + util.PadRight("DeletedAt", 21), + util.PadRight("TruncatedAt", 19), + "\n", + )) + for _, g := range resp.Groups { + fmt.Print( + util.PadRight(fmt.Sprint(g.ID), 10), + formatTimeStamp(g.StartTime), + " ", formatTimeStamp(g.EndTime), + " ", formatTimeStamp(g.DeletedAt), + " ", formatTimeStamp(g.TruncatedAt), + "\n", + ) + for _, shard := range g.Shards { + fmt.Print(color.CyanString(" |--Shard: %d\tNodes: [", shard.ID)) + first := true + for _, id := range shard.Nodes { + if n, ok := nodes[id]; ok { + if !first { + fmt.Print(color.CyanString(", ")) + } + fmt.Print(color.CyanString("%s", n.TcpAddr)) + first = false + } + } + fmt.Println(color.CyanString("]")) + } + if len(g.Shards) == 0 { + fmt.Println("No shard") + } + } + fmt.Println() + return nil +} + +func GetShard(addr, shard string) error { + var req controller.GetShardRequest + var resp controller.ShardResponse + respTyp := byte(controller.ResponseShard) + reqTyp := byte(controller.RequestShard) + id, err := strconv.ParseUint(shard, 10, 64) + if err != nil || id < 1 { + return errors.New("Please specify correct shard id") + } + req.ShardID = id + if err := RequestAndWaitResp(addr, reqTyp, respTyp, req, &resp); err != nil { + return err + } + if resp.Code != 0 { + return errors.New(resp.Msg) + } + + color.Set(color.Bold) + fmt.Println(color.GreenString("Shard:"), resp.ID) + color.Set(color.Bold) + fmt.Println(color.GreenString("Database:"), resp.DB) + color.Set(color.Bold) + fmt.Println(color.GreenString("Retention Policy:"), resp.Rp) + color.Set(color.Bold) + fmt.Println(color.GreenString("Begin:"), formatTimeStamp(resp.Begin)) + color.Set(color.Bold) + fmt.Println(color.GreenString("End:"), formatTimeStamp(resp.End)) + color.Set(color.Bold) + fmt.Println(color.GreenString("Truncated:"), formatTimeStamp(resp.Truncated)) + color.Set(color.Bold) + fmt.Print(color.GreenString("Nodes: ")) + fmt.Printf("%v\n", resp.Nodes) + fmt.Println() + return nil +} + func CopyShardStatus(addr string) error { var resp controller.CopyShardStatusResponse respTyp := byte(controller.ResponseCopyShardStatus) @@ -62,7 +192,12 @@ func CopyShardStatus(addr string) error { return err } - fmt.Printf("%+v\n", resp) + color.Set(color.Bold) + color.Green("Running Copy Tasks:\n") + for _, t := range resp.Tasks { + fmt.Print(t.ShardID, "\t", t.Database, "\t", t.Rp, "\t", t.CurrentSize, "/", t.TotalSize, "\t", t.Source, "\n") + } + fmt.Println() return nil } @@ -111,9 +246,9 @@ func RemoveShard(addr, shardID string) error { return nil } -func RemoveDataNode(addr string) error { +func RemoveDataNode(addr, removed_addr string) error { req := &controller.RemoveDataNodeRequest{ - DataNodeAddr: addr, + DataNodeAddr: removed_addr, } var resp controller.RemoveDataNodeResponse @@ -123,19 +258,76 @@ func RemoveDataNode(addr string) error { return err } + color.Set(color.Bold) + color.Green("Result: ") fmt.Println(resp.Msg) return nil } -func ShowDataNodes(addr string) error { +func freezeDataNode(addr, freezed_addr string, freeze bool) error { + req := &controller.FreezeDataNodeRequest{ + DataNodeAddr: freezed_addr, + Freeze: freeze, + } + + var resp controller.FreezeDataNodeResponse + respTyp := byte(controller.ResponseFreezeDataNode) + reqTyp := byte(controller.RequestFreezeDataNode) + if err := RequestAndWaitResp(addr, reqTyp, respTyp, req, &resp); err != nil { + return err + } + + color.Set(color.Bold) + color.Green("Result: ") + fmt.Println(resp.Msg) + return nil +} + +func FreezeDataNode(addr, freezed_addr string) error { + return freezeDataNode(addr, freezed_addr, true) +} + +func UnfreezeDataNode(addr, freezed_addr string) error { + return freezeDataNode(addr, freezed_addr, false) +} + +func getNodes(addr string) (map[uint64]controller.DataNode, error) { var resp controller.ShowDataNodesResponse respTyp := byte(controller.ResponseShowDataNodes) reqTyp := byte(controller.RequestShowDataNodes) if err := RequestAndWaitResp(addr, reqTyp, respTyp, struct{}{}, &resp); err != nil { - return err + return nil, err + } + nodes := make(map[uint64]controller.DataNode) + for _, n := range resp.DataNodes { + nodes[n.ID] = n } + return nodes, nil +} - fmt.Printf("msg:%s, data nodes:%+v\n", resp.Msg, resp.DataNodes) +func ShowDataNodes(addr string) error { + color.Set(color.Bold) + color.Green("Nodes:\n") + nodes, err := getNodes(addr) + if err != nil { + return err + } + ids := make([]uint64, 0, len(nodes)) + for _, n := range nodes { + ids = append(ids, n.ID) + } + sort.Slice(ids, func(i, j int) bool { + return ids[i] < ids[j] + }) + for _, id := range ids { + n := nodes[id] + fmt.Print(n.ID, "\thttp://", n.HttpAddr, "\ttcp://", n.TcpAddr) + if n.Freezed { + fmt.Print("\t(freezed)") + } + fmt.Print("\n") + } + fmt.Println() return nil } @@ -154,16 +346,16 @@ func RequestAndWaitResp(addr string, reqTyp, respTyp byte, req interface{}, resp return DecodeTLV(conn, respTyp, resp) } -func DecodeTLV(r io.Reader, expTyp byte, v interface{}) error { - typ, err := coordinator.ReadType(r) +func DecodeTLV(conn net.Conn, expTyp byte, v interface{}) error { + typ, err := coordinator.ReadType(conn) if err != nil { return err } if expTyp != typ { - return fmt.Errorf("invalid type, exp: %s, got: %s", expTyp, typ) + return fmt.Errorf("invalid type, exp: %d, got: %d", expTyp, typ) } - buf, err := coordinator.ReadLV(r) + buf, err := coordinator.ReadLV(conn, 4*time.Second) if err != nil { return err } @@ -185,6 +377,7 @@ func Dial(addr string) (net.Conn, error) { conn.Close() return nil, err } + conn.SetDeadline(time.Time{}) return conn, nil } diff --git a/cmd/influxd-ctl/action/util.go b/cmd/influxd-ctl/action/util.go new file mode 100644 index 0000000..ac052d3 --- /dev/null +++ b/cmd/influxd-ctl/action/util.go @@ -0,0 +1,12 @@ +package action + +import "time" + +const ( + TIME_FORMAT = "2006-01-02 15:04:05" +) + +func formatTimeStamp(millis int64) string { + t := time.Unix(millis/1000, (millis%1000)*1e6) + return t.Local().Format(TIME_FORMAT) +} diff --git a/cmd/influxd-ctl/command/command.go b/cmd/influxd-ctl/command/command.go index d83d79b..ef8eaff 100644 --- a/cmd/influxd-ctl/command/command.go +++ b/cmd/influxd-ctl/command/command.go @@ -1,122 +1,220 @@ package command import ( + "errors" "fmt" + "strconv" + "github.com/angopher/chronus/cmd/influxd-ctl/action" - "github.com/spf13/cobra" + "github.com/urfave/cli/v2" ) const ( defaultHost = "127.0.0.1:8088" ) -func NewCommand() *cobra.Command { - var copyShardCmd = &cobra.Command{ - Use: "copy-shard ", - Short: "copy shard", - Long: `copies a shard from a source data node to a destination data node.`, - Args: cobra.MinimumNArgs(3), - Run: func(cmd *cobra.Command, args []string) { - action.CopyShard(args[0], args[1], args[2]) - }, - } +var ( + DataNodeAddress string +) - var truncateCmd = &cobra.Command{ - Use: "truncate-shards [ip:port]", - Short: "truncates hot shards", - Long: `Truncates hot shards, that is, shards that cover the time range that includes the current time (now()).The truncate-shards command creates a new shard and the system writes all new points to that shard.`, - Args: cobra.RangeArgs(1, 2), - Run: func(cmd *cobra.Command, args []string) { - host := defaultHost - if len(args) == 2 { - host = args[1] - } - if err := action.TruncateShards(args[0], host); err != nil { - fmt.Println(err) - } - }, - } +func NodeCommand() *cli.Command { + return &cli.Command{ + Name: "node", + Usage: "node related operations", + Subcommands: []*cli.Command{ + { + Name: "list", + Usage: "show nodes in cluster", + Action: func(ctx *cli.Context) error { + if err := action.ShowDataNodes(DataNodeAddress); err != nil { + fmt.Println(err) + } + return nil + }, + }, { + Name: "freeze", + ArgsUsage: "freeze ", + Usage: "freeze specified node preventing creating new shard on", + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 1 { + return errors.New("Please specify node addr to be freezed") + } + if err := action.FreezeDataNode(DataNodeAddress, ctx.Args().Get(0)); err != nil { + fmt.Println(err) + } - var copyShardStatusCmd = &cobra.Command{ - Use: "copy-shard-status [ip:port]", - Short: "Displaying all in-progress copy-shard operations", - Long: `Shows all in-progress copy shard operations, including the shard’s source node, - destination node, database, retention policy, shard ID, total size, - current size, and the operation’s start time.`, - Args: cobra.RangeArgs(0, 1), - Run: func(cmd *cobra.Command, args []string) { - host := defaultHost - if len(args) == 1 { - host = args[0] - } - if err := action.CopyShardStatus(host); err != nil { - fmt.Println(err) - } - }, - } + return nil + }, + }, { + Name: "unfreeze", + ArgsUsage: "unfreeze ", + Usage: "unfreeze specified node", + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 1 { + return errors.New("Please specify node addr to unfreeze") + } + if err := action.UnfreezeDataNode(DataNodeAddress, ctx.Args().Get(0)); err != nil { + fmt.Println(err) + } - var killCopyShardCmd = &cobra.Command{ - Use: "kill-copy-shard ", - Short: "Aborts an in-progress copy-shard command.", - Long: "Aborts an in-progress copy-shard command.", - Args: cobra.MinimumNArgs(3), - Run: func(cmd *cobra.Command, args []string) { - if err := action.KillCopyShard(args[0], args[1], args[2]); err != nil { - fmt.Println(err) - } - }, - } + return nil + }, + }, { + Name: "remove", + ArgsUsage: "remove ", + Usage: "remove specified node from cluster", + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 1 { + return errors.New("Please specify node addr to be removed from cluster") + } + if err := action.RemoveDataNode(DataNodeAddress, ctx.Args().Get(0)); err != nil { + fmt.Println(err) + } - var removeShardCmd = &cobra.Command{ - Use: "remove-shard ", - Short: "Removes a shard from a data node. Removing a shard is an irrecoverable, destructive action; please be cautious with this command.", - Long: "Removes a shard from a data node. Removing a shard is an irrecoverable, destructive action; please be cautious with this command.", - Args: cobra.MinimumNArgs(2), - Run: func(cmd *cobra.Command, args []string) { - if err := action.RemoveShard(args[0], args[1]); err != nil { - fmt.Println(err) - } + return nil + }, + }, }, } +} - var removeDataNodeCmd = &cobra.Command{ - Use: "remove-data-node ", - Short: "Removes a data node from a cluster.", - Long: "Removes a data node from a cluster.", - Args: cobra.MinimumNArgs(1), - Run: func(cmd *cobra.Command, args []string) { - if err := action.RemoveDataNode(args[0]); err != nil { - fmt.Println(err) - } - }, - } +func ShardCommand() *cli.Command { + return &cli.Command{ + Name: "shard", + Usage: "shard related operations", + Subcommands: []*cli.Command{ + { + Name: "list", + ArgsUsage: "list ", + Usage: "show all shards of specified retention policy", + Description: fmt.Sprint( + "List all shards in specified retention policy.", + ), + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 2 { + return errors.New("Please specify database and retention policy") + } + if err := action.ListShard(DataNodeAddress, ctx.Args().Get(0), ctx.Args().Get(1)); err != nil { + fmt.Println(err) + } + return nil + + }, + }, { + Name: "node", + ArgsUsage: "node ", + Usage: "show all shards' id on specified node", + Description: fmt.Sprint( + "List all shards on node.", + ), + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 1 { + return errors.New("Please specify node id") + } + nodeId, _ := strconv.ParseUint(ctx.Args().Get(0), 10, 64) + if nodeId < 1 { + return errors.New("Please specify node id") + } + if err := action.ListShardOnNode(DataNodeAddress, nodeId); err != nil { + fmt.Println(err) + } + return nil - var showDataNodesCmd = &cobra.Command{ - Use: "show-data-nodes [ip:port]", - Short: "Show all data node from a cluster.", - Long: "Show all data node from a cluster.", - Args: cobra.RangeArgs(0, 1), - Run: func(cmd *cobra.Command, args []string) { - host := defaultHost - if len(args) == 1 { - host = args[0] - } - if err := action.ShowDataNodes(host); err != nil { - fmt.Println(err) - } + }, + }, { + Name: "info", + ArgsUsage: "info ", + Usage: "show information of specified shard", + Description: fmt.Sprint( + "Show information of specified shard.", + ), + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 1 { + return errors.New("Please specify shard id") + } + if err := action.GetShard(DataNodeAddress, ctx.Args().Get(0)); err != nil { + fmt.Println(err.Error()) + } + return nil + }, + }, { + Name: "status", + Usage: "show progress of copy-shard tasks", + Description: fmt.Sprint( + "Shows all in-progress copy shard operations, including the shard’s source node,\n", + "destination node, database, retention policy, shard ID, total size,\n", + "current size, and the operation’s start time.", + ), + Action: func(ctx *cli.Context) error { + if err := action.CopyShardStatus(DataNodeAddress); err != nil { + fmt.Println(err) + } + return nil + }, + }, { + Name: "copy", + Usage: "copy a shard to current node", + ArgsUsage: "copy ", + Description: "Copy a shard from a source data node to a current data node which is specified through -s", + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 2 { + return errors.New("Please specify source node and shard") + } + action.CopyShard(ctx.Args().Get(0), DataNodeAddress, ctx.Args().Get(1)) + return nil + }, + }, { + Name: "remove", + Usage: "remove a shard", + ArgsUsage: "remove ", + Description: fmt.Sprint( + "Removes a shard from current data node.\n", + "Removing a shard is an irrecoverable, destructive action;\n", + "Please be cautious with this command.", + ), + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 1 { + return errors.New("Please specify shard") + } + if err := action.RemoveShard(DataNodeAddress, ctx.Args().First()); err != nil { + fmt.Println(err) + } + return nil + }, + }, { + Name: "stop", + Usage: "stop a task of copy shard", + ArgsUsage: "stop ", + Description: "Stop a task of copy shard to current node which specified through -s.", + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 2 { + return errors.New("Please specify source node and shard") + } + if err := action.KillCopyShard(ctx.Args().Get(0), DataNodeAddress, ctx.Args().Get(1)); err != nil { + fmt.Println(err) + } + return nil + }, + }, { + Name: "truncate", + Usage: "truncates hot shards", + ArgsUsage: "truncate ", + Description: fmt.Sprint( + "Truncates hot shards, that is, shards that cover the time range\n", + "that includes the current time (now()).\n", + "The truncate-shards command creates a new shard and \n", + "the system writes all new points to that shard.", + ), + Action: func(ctx *cli.Context) error { + if ctx.Args().Len() < 1 { + return errors.New("delay seconds should be specified") + } + if err := action.TruncateShards(ctx.Args().Get(0), DataNodeAddress); err != nil { + fmt.Println(err) + } + return nil + }, + }, }, } - - var rootCmd = &cobra.Command{Use: "influxd-ctl"} - rootCmd.AddCommand( - copyShardCmd, - truncateCmd, - copyShardStatusCmd, - killCopyShardCmd, - removeShardCmd, - removeDataNodeCmd, - showDataNodesCmd, - ) - - return rootCmd } diff --git a/cmd/influxd-ctl/main.go b/cmd/influxd-ctl/main.go index 1482dfd..b156fed 100644 --- a/cmd/influxd-ctl/main.go +++ b/cmd/influxd-ctl/main.go @@ -1,10 +1,38 @@ package main import ( + "fmt" + "os" + "github.com/angopher/chronus/cmd/influxd-ctl/command" + "github.com/fatih/color" + "github.com/urfave/cli/v2" ) func main() { - command := command.NewCommand() - command.Execute() + app := &cli.App{} + app.Name = "influxd-ctl" + app.Usage = "Maintain the data nodes in cluster" + app.ExitErrHandler = func(ctx *cli.Context, err error) { + if err == nil { + return + } + color.Red(err.Error()) + fmt.Println() + } + + app.Commands = []*cli.Command{ + command.NodeCommand(), + command.ShardCommand(), + } + app.Flags = []cli.Flag{ + &cli.StringFlag{ + Name: "node", + Aliases: []string{"s"}, + Required: true, + Usage: "DataNode address in cluster, ip:port", + Destination: &command.DataNodeAddress, + }, + } + app.Run(os.Args) } diff --git a/cmd/influxd/run/command.go b/cmd/influxd/run/command.go index d6c3dce..c1a2276 100644 --- a/cmd/influxd/run/command.go +++ b/cmd/influxd/run/command.go @@ -6,7 +6,6 @@ import ( "fmt" "io" "io/ioutil" - "log" "net/http" _ "net/http/pprof" "os" @@ -15,6 +14,7 @@ import ( "strconv" "time" + "github.com/angopher/chronus/logging" "github.com/influxdata/influxdb/logger" "go.uber.org/zap" ) @@ -88,10 +88,14 @@ func (cmd *Command) Run(args ...string) error { return fmt.Errorf("%s. To generate a valid configuration file run `influxd config > influxdb.generated.conf`", err) } - var logErr error - if cmd.Logger, logErr = config.Logging.New(cmd.Stderr); logErr != nil { - // assign the default logger - cmd.Logger = logger.New(cmd.Stderr) + cmd.Logger, err = logging.InitialLogging(&logging.Config{ + Format: config.Logging.Format, + Level: config.Logging.Level.String(), + Dir: options.LogDir, + FileName: "influxd.log", + }) + if err != nil { + return fmt.Errorf("%s. Initialize logging failed", err) } // Attempt to run pprof on :6060 before startup if debug pprof enabled. @@ -115,11 +119,6 @@ func (cmd *Command) Run(args ...string) error { zap.String("version", runtime.Version()), zap.Int("maxprocs", runtime.GOMAXPROCS(0))) - // If there was an error on startup when creating the logger, output it now. - if logErr != nil { - cmd.Logger.Error("Unable to configure logger", zap.Error(logErr)) - } - // Write the PID file. if err := cmd.writePIDFile(options.PIDFile); err != nil { return fmt.Errorf("write pid file: %s", err) @@ -139,11 +138,10 @@ func (cmd *Command) Run(args ...string) error { Branch: cmd.Branch, Time: cmd.BuildTime, } - s, err := NewServer(config, buildInfo) + s, err := NewServer(config, buildInfo, cmd.Logger) if err != nil { return fmt.Errorf("create server: %s", err) } - s.Logger = cmd.Logger s.CPUProfile = options.CPUProfile s.MemProfile = options.MemProfile if err := s.Open(); err != nil { @@ -169,11 +167,10 @@ func (cmd *Command) Close() error { } func (cmd *Command) monitorServerErrors() { - logger := log.New(cmd.Stderr, "", log.LstdFlags) for { select { case err := <-cmd.Server.Err(): - logger.Println(err) + cmd.Logger.Error(err.Error()) case <-cmd.closing: return } @@ -198,6 +195,7 @@ func (cmd *Command) ParseFlags(args ...string) (Options, error) { _ = fs.String("hostname", "", "") fs.StringVar(&options.CPUProfile, "cpuprofile", "", "") fs.StringVar(&options.MemProfile, "memprofile", "", "") + fs.StringVar(&options.LogDir, "logdir", "", "Log to specified directory") fs.Usage = func() { fmt.Fprintln(cmd.Stderr, usage) } if err := fs.Parse(args); err != nil { return Options{}, err @@ -262,6 +260,8 @@ Usage: influxd run [flags] Write CPU profiling information to a file. -memprofile Write memory usage information to a file. + -logdir + Write logs to specified path ` // Options represents the command line options that can be parsed. @@ -270,6 +270,7 @@ type Options struct { PIDFile string CPUProfile string MemProfile string + LogDir string } // GetConfigPath returns the config path from the options. diff --git a/cmd/influxd/run/config.go b/cmd/influxd/run/config.go index f707a5d..b0a642f 100644 --- a/cmd/influxd/run/config.go +++ b/cmd/influxd/run/config.go @@ -4,11 +4,9 @@ import ( "fmt" "io/ioutil" "log" - "os" - "os/user" - "path/filepath" "regexp" "strings" + "time" "github.com/BurntSushi/toml" "github.com/influxdata/influxdb/logger" @@ -75,16 +73,24 @@ type Config struct { func NewConfig() *Config { c := &Config{} c.Meta = meta.NewConfig() + c.Meta.Dir = "./meta" c.Data = tsdb.NewConfig() + c.Data.Dir = "./data" + c.Data.WALDir = "./wal" + c.Data.QueryLogEnabled = false c.Coordinator = coordinator.NewConfig() + c.Coordinator.LogQueriesAfter = itoml.Duration(500 * time.Millisecond) c.Precreator = precreator.NewConfig() c.Monitor = monitor.NewConfig() c.Subscriber = subscriber.NewConfig() c.HTTPD = httpd.NewConfig() + c.HTTPD.AccessLogPath = "./logs/access.log" c.Logging = logger.NewConfig() c.HintedHandoff = hh.NewConfig() + c.HintedHandoff.Enabled = true + c.HintedHandoff.Dir = "./hh" c.Controller = controller.NewConfig() c.GraphiteInputs = []graphite.Config{graphite.NewConfig()} @@ -93,6 +99,7 @@ func NewConfig() *Config { c.UDPInputs = []udp.Config{udp.NewConfig()} c.ContinuousQuery = continuous_querier.NewConfig() + c.ContinuousQuery.RunInterval = itoml.Duration(time.Minute) c.Retention = retention.NewConfig() c.BindAddress = DefaultBindAddress @@ -103,20 +110,9 @@ func NewConfig() *Config { func NewDemoConfig() (*Config, error) { c := NewConfig() - var homeDir string - // By default, store meta and data files in current users home directory - u, err := user.Current() - if err == nil { - homeDir = u.HomeDir - } else if os.Getenv("HOME") != "" { - homeDir = os.Getenv("HOME") - } else { - return nil, fmt.Errorf("failed to determine current user for storage") - } - - c.Meta.Dir = filepath.Join(homeDir, ".influxdb/meta") - c.Data.Dir = filepath.Join(homeDir, ".influxdb/data") - c.Data.WALDir = filepath.Join(homeDir, ".influxdb/wal") + c.Meta.Dir = "./meta" + c.Data.Dir = "./data" + c.Data.WALDir = "./wal" return c, nil } diff --git a/cmd/influxd/run/server.go b/cmd/influxd/run/server.go index 61e6aee..fe374a0 100644 --- a/cmd/influxd/run/server.go +++ b/cmd/influxd/run/server.go @@ -30,9 +30,9 @@ import ( "github.com/influxdata/influxdb/services/storage" "github.com/influxdata/influxdb/services/subscriber" "github.com/influxdata/influxdb/services/udp" + "github.com/influxdata/influxdb/storage/reads" "github.com/influxdata/influxdb/tcp" "github.com/influxdata/influxdb/tsdb" - "github.com/influxdata/platform/storage/reads" client "github.com/influxdata/usage-client/v1" "go.uber.org/zap" @@ -45,6 +45,7 @@ import ( "github.com/angopher/chronus/services/controller" "github.com/angopher/chronus/services/hh" imeta "github.com/angopher/chronus/services/meta" + "github.com/angopher/chronus/x" ) var startTime time.Time @@ -73,7 +74,8 @@ type Server struct { BindAddress string Listener net.Listener - Logger *zap.Logger + Logger *zap.Logger + SugarLogger *zap.SugaredLogger Node *influxdb.Node ClusterMetaClient *coordinator.ClusterMetaClient @@ -101,6 +103,8 @@ type Server struct { CPUProfile string MemProfile string + clusterExecutor *coordinator.ClusterExecutor + // httpAPIAddr is the host:port combination for the main HTTP API for querying and writing data httpAPIAddr string @@ -122,7 +126,7 @@ func updateTLSConfig(into **tls.Config, with *tls.Config) { } // NewServer returns a new instance of Server built from a config. -func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { +func NewServer(c *Config, buildInfo *BuildInfo, logger *zap.Logger) (*Server, error) { // First grab the base tls config we will use for all clients and servers tlsConfig, err := c.TLS.Parse() if err != nil { @@ -186,7 +190,8 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { Node: node, - Logger: logger.New(os.Stderr), + Logger: logger, + SugarLogger: logger.Sugar(), ClusterMetaClient: coordinator.NewMetaClient(c.Meta, c.Coordinator, nodeID), @@ -205,23 +210,15 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { if err := s.ClusterMetaClient.Open(); err != nil { return nil, err } - - wait := s.ClusterMetaClient.WaitForDataChanged() - go s.ClusterMetaClient.RunSyncLoop() - //wait sync meta data from meta server - select { - case <-time.After(5 * time.Second): - //TODO: - panic("sync meta data failed") - case <-wait: - } + s.ClusterMetaClient.Start() // If we've already created a data node for our id, we're done n, err := s.ClusterMetaClient.DataNode(nodeID) if err != nil { + s.Logger.Warn(fmt.Sprintf("Node id(%d) from store can't be used, try to create new", nodeID), zap.Error(err)) n, err = s.ClusterMetaClient.CreateDataNode(s.httpAPIAddr, s.tcpAddr) if err != nil { - log.Printf("Unable to create data node. err: %s", err.Error()) + s.Logger.Warn(fmt.Sprint("Unable to create data node. err: ", err.Error())) return nil, err } } @@ -242,11 +239,28 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { s.Subscriber = subscriber.NewService(c.Subscriber) // Initialize shard writer - s.ShardWriter = coordinator.NewShardWriter(time.Duration(s.config.Coordinator.WriteTimeout), s.config.Coordinator.PoolMaxConnections) + s.ShardWriter = coordinator.NewShardWriter( + time.Duration(s.config.Coordinator.WriteTimeout), + coordinator.NewClientPool(func(nodeId uint64) (x.ConnPool, error) { + return x.NewBoundedPool( + 1, + 100, + time.Duration(s.config.Coordinator.PoolMaxIdleTimeout), + time.Duration(s.config.Coordinator.DailTimeout), + coordinator.NewClientConnFactory( + nodeId, + time.Duration(s.config.Coordinator.DailTimeout), + s.ClusterMetaClient, + ).Dial, + ) + }), + ) + s.ShardWriter.WithLogger(s.Logger) // Create the hinted handoff service s.HintedHandoff = hh.NewService(c.HintedHandoff, s.ShardWriter, s.ClusterMetaClient) s.HintedHandoff.Monitor = s.Monitor + s.HintedHandoff.WithLogger(s.Logger) // Initialize points writer. s.PointsWriter = coordinator.NewPointsWriter() @@ -257,10 +271,28 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { s.PointsWriter.HintedHandoff = s.HintedHandoff // Initialize cluster extecutor - clusterExecutor := coordinator.NewClusterExecutor(s.Node, s.TSDBStore, s.ClusterMetaClient, s.config.Coordinator) + clusterExecutor := coordinator.NewClusterExecutor( + s.Node, s.TSDBStore, + s.ClusterMetaClient, + coordinator.NewClientPool(func(nodeId uint64) (x.ConnPool, error) { + return x.NewBoundedPool( + x.Max(1, s.config.Coordinator.PoolMinStreamsPerNode), + s.config.Coordinator.PoolMaxStreamsPerNode, + time.Duration(s.config.Coordinator.PoolMaxIdleTimeout), + time.Duration(s.config.Coordinator.DailTimeout), + coordinator.NewClientConnFactory( + nodeId, + time.Duration(s.config.Coordinator.DailTimeout), + s.ClusterMetaClient, + ).Dial, + ) + }), + s.config.Coordinator, + ) clusterExecutor.WithLogger(s.Logger) // Initialize query executor. + s.clusterExecutor = clusterExecutor s.QueryExecutor = query.NewExecutor() s.QueryExecutor.StatementExecutor = &influxdb_coordinator.StatementExecutor{ MetaClient: s.ClusterMetaClient, @@ -446,15 +478,7 @@ func (s *Server) appendControllerService(c controller.Config) { srv.MetaClient = s.ClusterMetaClient srv.Node = s.Node srv.TSDBStore = s.TSDBStore - shardCopier := &controller.ShardCopier{ - Manager: controller.NewCopyManager(c.MaxShardCopyTasks), - MetaClient: s.ClusterMetaClient, - TSDBStore: s.TSDBStore, - Node: s.Node, - } - shardCopier.WithLogger(s.Logger) - srv.ShardCopier = shardCopier s.ControllerService = srv s.Services = append(s.Services, srv) } @@ -560,6 +584,8 @@ func (s *Server) Open() error { go s.startServerReporting() } + go s.poolStatsLoop() + return nil } @@ -669,6 +695,36 @@ func (s *Server) reportServer() { go cl.Save(usage) } +func dumpPoolStats(name string, sugar *zap.SugaredLogger, stats []coordinator.StatEntity) { + sugar.Info("Dump statistics of ", name) + sugar.Info("active/idle/capacity, success/cost(ms), failure/cost(ms), return/close") + for _, stat := range stats { + sugar.Infof("Node %d: %d/%d/%d, %d/%d, %d/%d, %d/%d", + stat.NodeId, + stat.Stat.Active, stat.Stat.Idle, stat.Stat.Capacity, + stat.Stat.GetSuccessCnt, stat.Stat.GetSuccessMillis, + stat.Stat.GetFailureCnt, stat.Stat.GetFailureMillis, + stat.Stat.ReturnCnt, stat.Stat.CloseCnt, + ) + } +} + +func (s *Server) poolStatsLoop() { + ticker := time.NewTicker(120 * time.Second) + +LOOP: + for { + select { + case <-ticker.C: + dumpPoolStats("remote executors", s.SugarLogger, s.clusterExecutor.RemoteNodeExecutor.Stats()) + dumpPoolStats("shard writers", s.SugarLogger, s.ShardWriter.Stats()) + case <-s.closing: + //shutdown + break LOOP + } + } +} + // Service represents a service attached to the server. type Service interface { WithLogger(log *zap.Logger) diff --git a/cmd/metad-ctl/.gitignore b/cmd/metad-ctl/.gitignore new file mode 100644 index 0000000..3cf3346 --- /dev/null +++ b/cmd/metad-ctl/.gitignore @@ -0,0 +1 @@ +metad-ctl diff --git a/cmd/metad-ctl/cmds/parse.go b/cmd/metad-ctl/cmds/parse.go new file mode 100644 index 0000000..adb79a1 --- /dev/null +++ b/cmd/metad-ctl/cmds/parse.go @@ -0,0 +1,68 @@ +package cmds + +import ( + "encoding/json" + "errors" + "strconv" + "strings" + + "github.com/angopher/chronus/raftmeta" + "github.com/urfave/cli/v2" +) + +var ( + FLAG_ADDR = &cli.StringFlag{ + Name: "metad", + Aliases: []string{"s"}, + Required: true, + Usage: "Node address in cluster, ip:port", + Destination: &MetadAddress, + } +) + +func parseNodeAddr(arg string) (string, error) { + if len(arg) < 6 || !strings.Contains(arg, ":") { + return "", errors.New("Incorrect address, should be ip:port") + } + return arg, nil +} + +func parseNodeId(arg string) (uint64, error) { + id, err := strconv.ParseUint(arg, 10, 64) + if err != nil { + return 0, err + } + if id < 1 { + err = errors.New("Node id should be positive") + return 0, err + } + return id, nil +} + +func parseNodeIdAndAddr(arg0, arg1 string) (id uint64, addr string, err error) { + id, err = parseNodeId(arg0) + if err != nil { + return + } + + addr, err = parseNodeAddr(arg1) + if err != nil { + return + } + + return +} + +func processResponse(data []byte) error { + resp := raftmeta.CommonResp{} + err := json.Unmarshal(data, &resp) + if err != nil { + return err + } + + if resp.RetCode != 0 { + return errors.New(resp.RetMsg) + } + + return nil +} diff --git a/cmd/metad-ctl/cmds/status.go b/cmd/metad-ctl/cmds/status.go new file mode 100644 index 0000000..8f3e450 --- /dev/null +++ b/cmd/metad-ctl/cmds/status.go @@ -0,0 +1,73 @@ +package cmds + +import ( + "encoding/json" + "errors" + "fmt" + "sort" + + "github.com/angopher/chronus/cmd/metad-ctl/util" + "github.com/angopher/chronus/raftmeta" + "github.com/fatih/color" + "github.com/urfave/cli/v2" +) + +func dumpStatus(resp *raftmeta.StatusClusterResp) { + color.Set(color.Bold) + color.Green("Cluster:\n") + fmt.Println("Leader:", resp.Leader) + fmt.Println("Term:", resp.Term) + fmt.Println("Committed:", resp.Commit) + fmt.Println("Applied:", resp.Applied) + fmt.Println() + + color.Set(color.Bold) + color.Yellow("Nodes:\n") + sort.Slice(resp.Nodes, func(i, j int) bool { + return resp.Nodes[i].ID < resp.Nodes[j].ID + }) + for _, n := range resp.Nodes { + fmt.Print(util.PadRight(fmt.Sprint(n.ID), 6)) + fmt.Print(" ", util.PadRight(n.Role, 15)) + fmt.Print(util.PadRight(n.Addr, 23)) + fmt.Print(util.PadRight(n.Progress, 15)) + fmt.Print(util.PadRight(fmt.Sprint(n.Match, "=>", n.Next), 20)) + if n.Vote > 0 { + fmt.Print("Vote(", n.Vote, ")") + } + fmt.Println() + } +} + +func StatusCommand() *cli.Command { + return &cli.Command{ + Name: "status", + Usage: "status of cluster", + Description: "Get the cluster status", + Action: clusterStatus, + Flags: []cli.Flag{FLAG_ADDR}, + } +} + +func clusterStatus(ctx *cli.Context) (err error) { + resp := &raftmeta.StatusClusterResp{} + data, err := util.GetRequest(fmt.Sprint("http://", MetadAddress, "/status_cluster")) + if err != nil { + goto ERR + } + + err = json.Unmarshal(data, resp) + if err != nil { + goto ERR + } + if resp.RetCode != 0 { + err = errors.New(resp.RetMsg) + goto ERR + } + + dumpStatus(resp) + return nil + +ERR: + return err +} diff --git a/cmd/metad-ctl/cmds/storage.go b/cmd/metad-ctl/cmds/storage.go new file mode 100644 index 0000000..d48231d --- /dev/null +++ b/cmd/metad-ctl/cmds/storage.go @@ -0,0 +1,97 @@ +package cmds + +import ( + "errors" + "fmt" + "io/ioutil" + "os" + "path/filepath" + "sort" + + "github.com/angopher/chronus/cmd/metad-ctl/util" + "github.com/angopher/chronus/raftmeta" + "github.com/dgraph-io/badger/v2" + "github.com/dgraph-io/badger/v2/table" + "github.com/dustin/go-humanize" + "github.com/fatih/color" + "github.com/urfave/cli/v2" +) + +func dumpStatus1(resp *raftmeta.StatusClusterResp) { + color.Set(color.Bold) + color.Green("Cluster:\n") + fmt.Println("Leader:", resp.Leader) + fmt.Println("Term:", resp.Term) + fmt.Println("Committed:", resp.Commit) + fmt.Println("Applied:", resp.Applied) + fmt.Println() + + color.Set(color.Bold) + color.Yellow("Nodes:\n") + sort.Slice(resp.Nodes, func(i, j int) bool { + return resp.Nodes[i].ID < resp.Nodes[j].ID + }) + for _, n := range resp.Nodes { + fmt.Print(util.PadRight(fmt.Sprint(n.ID), 6)) + fmt.Print(" ", util.PadRight(n.Role, 15)) + fmt.Print(util.PadRight(n.Addr, 23)) + fmt.Print(util.PadRight(n.Progress, 15)) + fmt.Print(util.PadRight(fmt.Sprint(n.Match, "=>", n.Next), 20)) + if n.Vote > 0 { + fmt.Print("Vote(", n.Vote, ")") + } + fmt.Println() + } +} + +func StorageCommand() *cli.Command { + return &cli.Command{ + Name: "storage", + Usage: "View storage info", + Description: "Get the storage info", + Action: storageInfo, + } +} + +func storageInfo(ctx *cli.Context) (err error) { + if ctx.Args().Len() < 1 { + return errors.New("Usage: metad-ctl storage ") + } + + walDir := ctx.Args().First() + fmt.Println("Dump information of", walDir) + + fileinfos, err := ioutil.ReadDir(walDir) + if err != nil { + return err + } + fmt.Println("Collect files ...") + fileinfoByName := make(map[string]os.FileInfo) + for _, info := range fileinfos { + fileinfoByName[info.Name()] = info + } + + fmt.Println("Reading", badger.ManifestFilename, "...") + fp, err := os.Open(filepath.Join(walDir, badger.ManifestFilename)) + if err != nil { + return err + } + defer fp.Close() + + manifest, _, err := badger.ReplayManifestFile(fp) + fmt.Println() + color.Set(color.Bold) + color.Green("Disk space cost by level:") + for level, lm := range manifest.Levels { + sz := int64(0) + for id := range lm.Tables { + tableFile := table.IDToFilename(id) + if info, ok := fileinfoByName[tableFile]; ok { + sz += info.Size() + } + } + fmt.Print("Level ", level, ": ", humanize.Bytes(uint64(sz)), "\n") + } + + return nil +} diff --git a/cmd/metad-ctl/cmds/update.go b/cmd/metad-ctl/cmds/update.go new file mode 100644 index 0000000..9a00618 --- /dev/null +++ b/cmd/metad-ctl/cmds/update.go @@ -0,0 +1,146 @@ +package cmds + +import ( + "errors" + "fmt" + "net/url" + + "github.com/angopher/chronus/cmd/metad-ctl/util" + "github.com/fatih/color" + "github.com/urfave/cli/v2" +) + +func AddCommand() *cli.Command { + return &cli.Command{ + Name: "add", + Usage: "Add node to cluster", + Description: "Introdue new node to cluster should follow two phases operation:\n 1. Add node to configuration using `metad-ctl add`\n 2. Boot up new node with confugration up to date", + ArgsUsage: " ", + Action: clusterAdd, + Flags: []cli.Flag{FLAG_ADDR}, + } +} + +func UpdateCommand() *cli.Command { + return &cli.Command{ + Name: "update", + Usage: "Update address of node in cluster", + Description: "Update address of a node which is already in cluster should follow two phases operation:\n 1. Stop the node\n 2. Update address\n 3. Boot up node with confugration up to date", + ArgsUsage: " ", + Action: clusterUpdate, + Flags: []cli.Flag{FLAG_ADDR}, + } +} + +func RemoveCommand() *cli.Command { + return &cli.Command{ + Name: "remove", + Usage: "Remove node from cluster", + Description: "Remove specified node from config.", + ArgsUsage: "", + Action: clusterRemove, + Flags: []cli.Flag{FLAG_ADDR}, + } +} + +func clusterAdd(ctx *cli.Context) (err error) { + var ( + id uint64 + addr string + data []byte + ) + if ctx.Args().Len() < 2 { + err = errors.New("Please specify node-id and address") + goto ERR + } + id, addr, err = parseNodeIdAndAddr(ctx.Args().Get(0), ctx.Args().Get(1)) + if err != nil { + goto ERR + } + + data, err = util.PostRequestJSON(fmt.Sprint("http://", MetadAddress, "/update_cluster?op=add"), map[string]interface{}{ + "ID": id, + "Addr": addr, + }) + if err != nil { + goto ERR + } + err = processResponse(data) + if err != nil { + goto ERR + } + + color.Green("Success") + return nil + +ERR: + return err +} + +func clusterUpdate(ctx *cli.Context) (err error) { + var ( + id uint64 + addr string + data []byte + ) + if ctx.Args().Len() < 2 { + err = errors.New("Please specify node-id and address") + goto ERR + } + id, addr, err = parseNodeIdAndAddr(ctx.Args().Get(0), ctx.Args().Get(1)) + if err != nil { + goto ERR + } + + data, err = util.PostRequestJSON(fmt.Sprint("http://", MetadAddress, "/update_cluster?op=update"), map[string]interface{}{ + "ID": id, + "Addr": addr, + }) + if err != nil { + goto ERR + } + err = processResponse(data) + if err != nil { + goto ERR + } + + color.Green("Success") + return nil + +ERR: + return err +} + +func clusterRemove(ctx *cli.Context) (err error) { + var ( + id uint64 + data []byte + ) + + if ctx.Args().Len() < 1 { + err = errors.New("Please specify node-id") + goto ERR + } + + id, err = parseNodeId(ctx.Args().Get(0)) + if err != nil { + goto ERR + } + + data, err = util.PostRequest(fmt.Sprint("http://", MetadAddress, "/update_cluster?op=remove"), url.Values{ + "node_id": {fmt.Sprint(id)}, + }) + if err != nil { + goto ERR + } + err = processResponse(data) + if err != nil { + goto ERR + } + + color.Green("Success") + return nil + +ERR: + return err +} diff --git a/cmd/metad-ctl/cmds/var.go b/cmd/metad-ctl/cmds/var.go new file mode 100644 index 0000000..3dd176b --- /dev/null +++ b/cmd/metad-ctl/cmds/var.go @@ -0,0 +1,3 @@ +package cmds + +var MetadAddress string diff --git a/cmd/metad-ctl/main.go b/cmd/metad-ctl/main.go new file mode 100644 index 0000000..edaec7a --- /dev/null +++ b/cmd/metad-ctl/main.go @@ -0,0 +1,32 @@ +package main + +import ( + "fmt" + "os" + + "github.com/angopher/chronus/cmd/metad-ctl/cmds" + "github.com/fatih/color" + "github.com/urfave/cli/v2" +) + +func main() { + app := &cli.App{} + app.Name = "metad-ctl" + app.Usage = "Maintain the metad cluster" + app.ExitErrHandler = func(ctx *cli.Context, err error) { + if err == nil { + return + } + color.Red(err.Error()) + fmt.Println() + } + + app.Commands = []*cli.Command{ + cmds.StatusCommand(), + cmds.AddCommand(), + cmds.UpdateCommand(), + cmds.RemoveCommand(), + cmds.StorageCommand(), + } + app.Run(os.Args) +} diff --git a/cmd/metad-ctl/util/request.go b/cmd/metad-ctl/util/request.go new file mode 100644 index 0000000..3e873cc --- /dev/null +++ b/cmd/metad-ctl/util/request.go @@ -0,0 +1,77 @@ +package util + +import ( + "bytes" + "encoding/json" + "io/ioutil" + "net" + "net/http" + "net/url" + "time" +) + +var client = http.Client{ + Transport: &http.Transport{ + Proxy: http.ProxyFromEnvironment, + DialContext: (&net.Dialer{ + Timeout: 30 * time.Second, + KeepAlive: 30 * time.Second, + DualStack: true, + }).DialContext, + ForceAttemptHTTP2: true, + MaxIdleConns: 10, + IdleConnTimeout: 90 * time.Second, + TLSHandshakeTimeout: 10 * time.Second, + ExpectContinueTimeout: 1 * time.Second, + }, +} + +func bodyData(resp *http.Response) ([]byte, error) { + defer resp.Body.Close() + resData, err := ioutil.ReadAll(resp.Body) + if err != nil { + return nil, err + } + return resData, nil +} + +func execute(req *http.Request) ([]byte, error) { + resp, err := client.Do(req) + if err != nil { + return nil, err + } + return bodyData(resp) +} + +func GetRequest(url string) ([]byte, error) { + req, err := http.NewRequest("GET", url, nil) + if err != nil { + return nil, err + } + + return execute(req) +} + +func PostRequest(urlStr string, data url.Values) ([]byte, error) { + resp, err := client.PostForm(urlStr, data) + if err != nil { + return nil, err + } + + return bodyData(resp) +} + +func PostRequestJSON(url string, obj interface{}) ([]byte, error) { + data, err := json.Marshal(obj) + if err != nil { + return nil, err + } + + req, err := http.NewRequest("POST", url, bytes.NewReader(data)) + if err != nil { + return nil, err + } + req.Header.Set("Content-Type", "application/json") + + return execute(req) +} diff --git a/cmd/metad-ctl/util/string.go b/cmd/metad-ctl/util/string.go new file mode 100644 index 0000000..ade40f2 --- /dev/null +++ b/cmd/metad-ctl/util/string.go @@ -0,0 +1,22 @@ +package util + +import ( + "fmt" + "strings" +) + +func PadLeft(str string, width int) string { + if len(str) >= width { + return str + } + + return fmt.Sprint(strings.Repeat(" ", width-len(str)), str) +} + +func PadRight(str string, width int) string { + if len(str) >= width { + return str + } + + return fmt.Sprint(str, strings.Repeat(" ", width-len(str))) +} diff --git a/cmd/metad-ctl/util/string_test.go b/cmd/metad-ctl/util/string_test.go new file mode 100644 index 0000000..5705a16 --- /dev/null +++ b/cmd/metad-ctl/util/string_test.go @@ -0,0 +1,31 @@ +package util + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestPadLeft(t *testing.T) { + assert.Equal(t, " 123", PadLeft("123", 6)) + assert.Equal(t, "123", PadLeft("123", 3)) + assert.Equal(t, "123", PadLeft("123", 2)) + assert.Equal(t, "123", PadLeft("123", 0)) + assert.Equal(t, "123", PadLeft("123", -1)) + assert.Equal(t, " 123", PadLeft("123", 4)) + + assert.Equal(t, " ", PadLeft("", 1)) + assert.Equal(t, " ", PadLeft("", 3)) +} + +func TestPadRight(t *testing.T) { + assert.Equal(t, "123 ", PadRight("123", 6)) + assert.Equal(t, "123", PadRight("123", 3)) + assert.Equal(t, "123", PadRight("123", 2)) + assert.Equal(t, "123", PadRight("123", 0)) + assert.Equal(t, "123", PadRight("123", -1)) + assert.Equal(t, "123 ", PadRight("123", 4)) + + assert.Equal(t, " ", PadRight("", 1)) + assert.Equal(t, " ", PadRight("", 3)) +} diff --git a/cmd/metad/main.go b/cmd/metad/main.go index efb4a20..e31be32 100644 --- a/cmd/metad/main.go +++ b/cmd/metad/main.go @@ -3,43 +3,92 @@ package main import ( "flag" "fmt" + "os" + "github.com/BurntSushi/toml" + "github.com/angopher/chronus/logging" "github.com/angopher/chronus/raftmeta" imeta "github.com/angopher/chronus/services/meta" "github.com/angopher/chronus/x" - "github.com/influxdata/influxdb/logger" "github.com/influxdata/influxdb/services/meta" - "os" + "go.etcd.io/etcd/raft/raftpb" + "go.uber.org/zap" ) func main() { - configFile := flag.String("config", "", "-config config_file") - flag.Parse() + f := flag.NewFlagSet("metad", flag.ExitOnError) + configFile := f.String("config", "", "Specify config file") + dumpFile := f.String("dump", "", "Boot and dump the snapshot to a file specified") + restoreFile := f.String("restore", "", "Boot and restore data from the snapshot specified") + showSample := f.Bool("sample", false, "Show sample configuration") + f.Parse(os.Args[1:]) config := raftmeta.NewConfig() + if *showSample { + toml.NewEncoder(os.Stdout).Encode(&config) + fmt.Println() + return + } + if *configFile != "" { x.Check((&config).FromTomlFile(*configFile)) } else { - toml.NewEncoder(os.Stdout).Encode(&config) + f.Usage() return } - fmt.Printf("config:%+v\n", config) - metaCli := imeta.NewClient(&meta.Config{ RetentionAutoCreate: config.RetentionAutoCreate, LoggingEnabled: true, }) + log, err := logging.InitialLogging(&logging.Config{ + Format: config.LogFormat, + Level: config.LogLevel, + Dir: config.LogDir, + FileName: "metad.log", + }) + if err != nil { + fmt.Fprintln(os.Stderr, "Error to initialize logging", err) + return + } - log := logger.New(os.Stderr) + suger := log.Sugar() + suger.Debug("config: %+v", config) metaCli.WithLogger(log) - err := metaCli.Open() + err = metaCli.Open() x.Check(err) - node := raftmeta.NewRaftNode(config) - node.MetaCli = metaCli - node.WithLogger(log) + node := raftmeta.NewRaftNode(config, log) + node.MetaStore = metaCli + + // dump only + if *dumpFile != "" { + err := node.Dump(*dumpFile) + if err != nil { + fmt.Println("Dump to file error:", err) + return + } + fmt.Println("Dumped to", *dumpFile) + return + } + + // restore + if *restoreFile != "" { + // set conf state first + var ids []uint64 + for _, n := range node.Config.Peers { + ids = append(ids, n.RaftId) + } + node.SetConfState(&raftpb.ConfState{ + Voters: ids, + }) + err = node.Restore(*restoreFile) + if err != nil { + node.Logger.Warn("Restore from file failed", zap.Error(err)) + return + } + } t := raftmeta.NewTransport() t.WithLogger(log) diff --git a/cmd/parse_test.go b/cmd/parse_test.go new file mode 100644 index 0000000..6ada1a6 --- /dev/null +++ b/cmd/parse_test.go @@ -0,0 +1,33 @@ +package cmd + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestParseCommandName(t *testing.T) { + name, args := ParseCommandName([]string{"-wrong", "test"}) + assert.Empty(t, name) + assert.Equal(t, 2, len(args)) + + name, args = ParseCommandName([]string{}) + assert.Empty(t, name) + assert.Equal(t, 0, len(args)) + + name, args = ParseCommandName([]string{"-h", "config"}) + assert.Equal(t, "help", name) + assert.Equal(t, 1, len(args)) + assert.Equal(t, "config", args[0]) + + name, args = ParseCommandName([]string{"--help", "config"}) + assert.Equal(t, "help", name) + assert.Equal(t, 1, len(args)) + assert.Equal(t, "config", args[0]) + + name, args = ParseCommandName([]string{"config", "-c", "a.yml"}) + assert.Equal(t, "config", name) + assert.Equal(t, 2, len(args)) + assert.Equal(t, "-c", args[0]) + assert.Equal(t, "a.yml", args[1]) +} diff --git a/coordinator/client_pool.go b/coordinator/client_pool.go index 3bb5041..f3f7e97 100644 --- a/coordinator/client_pool.go +++ b/coordinator/client_pool.go @@ -1,57 +1,194 @@ package coordinator import ( + "fmt" "net" + "sort" "sync" + "time" - "gopkg.in/fatih/pool.v2" + "github.com/angopher/chronus/x" + "github.com/influxdata/influxdb/services/meta" ) -type clientPool struct { - mu sync.RWMutex - pool map[uint64]pool.Pool +type PoolFactory func(nodeId uint64) (x.ConnPool, error) + +type ClientPool struct { + closer chan int + wg sync.WaitGroup + + mu sync.RWMutex + pools map[uint64]*poolEntry + factory PoolFactory } -func newClientPool() *clientPool { - return &clientPool{ - pool: make(map[uint64]pool.Pool), - } +type StatEntity struct { + NodeId uint64 + Stat x.PoolStatistics } -func (c *clientPool) setPool(nodeID uint64, p pool.Pool) { - c.mu.Lock() - c.pool[nodeID] = p - c.mu.Unlock() +type poolEntry struct { + pool x.ConnPool + lastUse time.Time } -func (c *clientPool) getPool(nodeID uint64) (pool.Pool, bool) { - c.mu.RLock() - p, ok := c.pool[nodeID] - c.mu.RUnlock() - return p, ok +func NewClientPool(factory PoolFactory) *ClientPool { + pool := &ClientPool{ + pools: make(map[uint64]*poolEntry), + closer: make(chan int), + factory: factory, + } + + go pool.idleCheckLoop() + return pool } -func (c *clientPool) size() int { - c.mu.RLock() +func (clientPool *ClientPool) Len() int { + clientPool.mu.RLock() var size int - for _, p := range c.pool { - size += p.Len() + for _, p := range clientPool.pools { + size += p.pool.Len() } - c.mu.RUnlock() + clientPool.mu.RUnlock() return size } -func (c *clientPool) conn(nodeID uint64) (net.Conn, error) { - c.mu.RLock() - conn, err := c.pool[nodeID].Get() - c.mu.RUnlock() - return conn, err +func (clientPool *ClientPool) Total() int { + clientPool.mu.RLock() + var size int + for _, p := range clientPool.pools { + size += p.pool.Total() + } + clientPool.mu.RUnlock() + return size +} + +func (clientPool *ClientPool) idleCheckOnce() { + clientPool.mu.Lock() + defer clientPool.mu.Unlock() + + now := time.Now() + var removed []uint64 + for nodeId, entry := range clientPool.pools { + if now.Sub(entry.lastUse) <= 600*time.Second { + continue + } + // close it + removed = append(removed, nodeId) + } + for _, id := range removed { + if entry, ok := clientPool.pools[id]; ok { + entry.pool.Close() + delete(clientPool.pools, id) + } + } +} + +func (clientPool *ClientPool) idleCheckLoop() { + clientPool.wg.Add(1) + defer clientPool.wg.Done() + + ticker := time.NewTicker(60 * time.Second) + defer ticker.Stop() + for { + select { + case <-ticker.C: + clientPool.idleCheckOnce() + case <-clientPool.closer: + return + } + } +} + +func (clientPool *ClientPool) GetConn(nodeID uint64) (x.PooledConn, error) { + clientPool.mu.RLock() + if entry, ok := clientPool.pools[nodeID]; ok { + entry.lastUse = time.Now() + clientPool.mu.RUnlock() + return entry.pool.Get() + } + clientPool.mu.RUnlock() + // switch to write lock + clientPool.mu.Lock() + defer clientPool.mu.Unlock() + // create new pool + pool, err := clientPool.factory(nodeID) + if err != nil { + return nil, err + } + clientPool.pools[nodeID] = &poolEntry{ + lastUse: time.Now(), + pool: pool, + } + return pool.Get() +} + +// Dump statistics of connection pools in this client +func (clientPool *ClientPool) Stat() []StatEntity { + clientPool.mu.RLock() + defer clientPool.mu.RUnlock() + + stats := make([]StatEntity, 0, len(clientPool.pools)) + for nodeId, pool := range clientPool.pools { + stats = append(stats, StatEntity{ + Stat: pool.pool.Statistics(), + NodeId: nodeId, + }) + } + sort.Slice(stats, func(i, j int) bool { + return stats[i].NodeId < stats[j].NodeId + }) + + return stats +} + +func (clientPool *ClientPool) close() { + clientPool.mu.Lock() + defer clientPool.mu.Unlock() + for _, entry := range clientPool.pools { + entry.pool.Close() + } + clientPool.pools = nil +} + +type ClientConnFactory struct { + nodeId uint64 + timeout time.Duration + + metaClient interface { + DataNode(id uint64) (ni *meta.NodeInfo, err error) + } +} + +func NewClientConnFactory(nodeId uint64, timeout time.Duration, metaClient MetaClient) *ClientConnFactory { + return &ClientConnFactory{ + nodeId: nodeId, + timeout: timeout, + metaClient: metaClient, + } } -func (c *clientPool) close() { - c.mu.Lock() - for _, p := range c.pool { - p.Close() +func (c *ClientConnFactory) Dial() (net.Conn, error) { + ni, err := c.metaClient.DataNode(c.nodeId) + if err != nil { + return nil, err + } + + if ni == nil { + return nil, fmt.Errorf("node %d does not exist", c.nodeId) } - c.mu.Unlock() + + conn, err := net.DialTimeout("tcp", ni.TCPHost, c.timeout) + if err != nil { + return nil, err + } + + // Write a marker byte for cluster messages. + _, err = conn.Write([]byte{MuxHeader}) + if err != nil { + conn.Close() + return nil, err + } + + return conn, nil } diff --git a/coordinator/cluster_executor.go b/coordinator/cluster_executor.go index 88963e8..b1198ad 100644 --- a/coordinator/cluster_executor.go +++ b/coordinator/cluster_executor.go @@ -3,7 +3,6 @@ package coordinator import ( "context" "fmt" - "math/rand" "sort" "sync" "time" @@ -35,23 +34,26 @@ type ClusterExecutor struct { Logger *zap.Logger } -func NewClusterExecutor(n *influxdb.Node, s TSDBStore, m MetaClient, Config Config) *ClusterExecutor { - return &ClusterExecutor{ +func NewClusterExecutor(n *influxdb.Node, s TSDBStore, m MetaClient, pool *ClientPool, Config Config) *ClusterExecutor { + executor := &ClusterExecutor{ Node: n, TSDBStore: s, MetaClient: m, RemoteNodeExecutor: &remoteNodeExecutor{ - MetaClient: m, + ClientPool: pool, DailTimeout: time.Duration(Config.DailTimeout), ShardReaderTimeout: time.Duration(Config.ShardReaderTimeout), ClusterTracing: Config.ClusterTracing, }, Logger: zap.NewNop(), } + executor.RemoteNodeExecutor.WithLogger(executor.Logger) + return executor } func (me *ClusterExecutor) WithLogger(log *zap.Logger) { me.Logger = log.With(zap.String("service", "ClusterExecutor")) + me.RemoteNodeExecutor.WithLogger(log) } func (me *ClusterExecutor) ExecuteStatement(stmt influxql.Statement, ctx *query.ExecutionContext) error { @@ -65,12 +67,13 @@ func (me *ClusterExecutor) ExecuteStatement(stmt influxql.Statement, ctx *query. return err } - nodes := NewNodeIdsByNodes(nodeInfos) + nodes := toNodeIds(nodeInfos) results := make(map[uint64]*Result) var mutex sync.Mutex var wg sync.WaitGroup - fn := func(nodeId uint64) { +LOOP: + for _, nodeId := range nodes { host := "unkown" node, _ := me.MetaClient.DataNode(nodeId) if node != nil { @@ -80,17 +83,18 @@ func (me *ClusterExecutor) ExecuteStatement(stmt influxql.Statement, ctx *query. switch t := stmt.(type) { case *influxql.KillQueryStatement: if t.Host != "" && t.Host != host { - return + // not this node + continue LOOP } } wg.Add(1) - go func() { - defer wg.Add(-1) + go func(curNodeId uint64) { + defer wg.Done() var err error var qr *query.Result - if nodeId == me.Node.ID { + if curNodeId == me.Node.ID { recvCtx := &query.ExecutionContext{ Context: context.Background(), Results: make(chan *query.Result, 1), @@ -100,7 +104,7 @@ func (me *ClusterExecutor) ExecuteStatement(stmt influxql.Statement, ctx *query. qr = <-recvCtx.Results } } else { - qr, err = me.RemoteNodeExecutor.TaskManagerStatement(nodeId, stmt) + qr, err = me.RemoteNodeExecutor.TaskManagerStatement(curNodeId, stmt) } if qr != nil && len(qr.Series) > 0 { @@ -111,12 +115,10 @@ func (me *ClusterExecutor) ExecuteStatement(stmt influxql.Statement, ctx *query. } mutex.Lock() - results[nodeId] = &Result{qr: qr, err: err} - mutex.Unlock() - }() + defer mutex.Unlock() + results[curNodeId] = &Result{qr: qr, err: err} + }(nodeId) } - - nodes.Apply(fn) wg.Wait() //merge result @@ -168,32 +170,29 @@ func (me *ClusterExecutor) SeriesCardinality(database string) (int64, error) { if err != nil { return -1, err } - nodes := NewNodeIdsByShards(shards) + nodes := getAllRelatedNodes(shards) results := make(map[uint64]*Result) var mutex sync.Mutex var wg sync.WaitGroup - fn := func(nodeId uint64) { + for _, nodeId := range nodes { wg.Add(1) - go func() { - defer wg.Add(-1) + go func(curNodeId uint64) { + defer wg.Done() var n int64 var err error - if nodeId == me.Node.ID { + if curNodeId == me.Node.ID { n, err = me.TSDBStore.SeriesCardinality(database) } else { - n, err = me.RemoteNodeExecutor.SeriesCardinality(nodeId, database) + n, err = me.RemoteNodeExecutor.SeriesCardinality(curNodeId, database) } mutex.Lock() - results[nodeId] = &Result{n: n, err: err} - mutex.Unlock() - return - }() + defer mutex.Unlock() + results[curNodeId] = &Result{n: n, err: err} + }(nodeId) } - - nodes.Apply(fn) wg.Wait() var sum int64 @@ -233,32 +232,29 @@ func (me *ClusterExecutor) MeasurementNames(auth query.Authorizer, database stri if err != nil { return nil, err } - nodes := NewNodeIdsByShards(shards) + nodes := getAllRelatedNodes(shards) results := make(map[uint64]*Result) var mutex sync.Mutex var wg sync.WaitGroup - fn := func(nodeId uint64) { + for _, nodeId := range nodes { wg.Add(1) - go func() { - defer wg.Add(-1) + go func(curNodeId uint64) { + defer wg.Done() var names [][]byte var err error - if nodeId == me.Node.ID { + if curNodeId == me.Node.ID { names, err = me.TSDBStore.MeasurementNames(auth, database, cond) } else { - names, err = me.RemoteNodeExecutor.MeasurementNames(nodeId, database, cond) + names, err = me.RemoteNodeExecutor.MeasurementNames(curNodeId, database, cond) } mutex.Lock() - results[nodeId] = &Result{names: names, err: err} - mutex.Unlock() - return - }() + defer mutex.Unlock() + results[curNodeId] = &Result{names: names, err: err} + }(nodeId) } - - nodes.Apply(fn) wg.Wait() uniq := make(map[string]struct{}) @@ -272,7 +268,7 @@ func (me *ClusterExecutor) MeasurementNames(auth query.Authorizer, database stri } strNames := make([]string, 0, len(uniq)) - for name, _ := range uniq { + for name := range uniq { strNames = append(strNames, name) } sort.Sort(StringSlice(strNames)) @@ -296,33 +292,20 @@ func (me *ClusterExecutor) TagValues(auth query.Authorizer, ids []uint64, cond i return nil, err } - n2s := NewNode2ShardIDs(me.MetaClient, me.Node, shards) - result := make(map[uint64]*TagValuesResult) - - var mutex sync.Mutex - var wg sync.WaitGroup - fn := func(nodeId uint64, shardIDs []uint64) { - wg.Add(1) - go func() { - defer wg.Add(-1) + n2s := PlanNodes(me.Node.ID, shards, nil) - var tagValues []tsdb.TagValues - var err error - if nodeId == me.Node.ID { - tagValues, err = me.TSDBStore.TagValues(auth, shardIDs, cond) - } else { - tagValues, err = me.RemoteNodeExecutor.TagValues(nodeId, shardIDs, cond) - } - - mutex.Lock() - result[nodeId] = &TagValuesResult{values: tagValues, err: err} - mutex.Unlock() - return - }() + fn := func(nodeId uint64, shards []meta.ShardInfo) (result interface{}, err error) { + var tagValues []tsdb.TagValues + shardIDs := toShardIDs(shards) + if nodeId == me.Node.ID { + tagValues, err = me.TSDBStore.TagValues(auth, shardIDs, cond) + } else { + tagValues, err = me.RemoteNodeExecutor.TagValues(nodeId, shardIDs, cond) + } + result = &TagValuesResult{values: tagValues, err: err} + return } - - n2s.Apply(fn) - wg.Wait() + result, err := n2s.ExecuteWithRetry(fn) extractKeyFn := func(kv tsdb.KeyValue) string { //TODO: @@ -330,7 +313,11 @@ func (me *ClusterExecutor) TagValues(auth query.Authorizer, ids []uint64, cond i } uniq := make(map[string]map[string]tsdb.KeyValue) - for nodeId, r := range result { + for nodeId, t := range result { + r, ok := t.(*TagValuesResult) + if !ok { + continue + } if r.err != nil { return nil, fmt.Errorf("TagValues fail, nodeId %d, err:%s", nodeId, r.err) } @@ -363,78 +350,78 @@ func (me *ClusterExecutor) TagValues(auth query.Authorizer, ids []uint64, cond i return tagValues, nil } +func (me *ClusterExecutor) createLocalIteratorfunc( + localCtx context.Context, + m *influxql.Measurement, + shardIDs []uint64, + opt query.IteratorOptions, +) (itr query.Iterator, err error) { + span := tracing.SpanFromContext(localCtx) + if span != nil { + span = span.StartSpan(fmt.Sprintf("local_node_id: %d", me.Node.ID)) + defer span.Finish() + + localCtx = tracing.NewContextWithSpan(localCtx, span) + } + sg := me.TSDBStore.ShardGroup(shardIDs) + if m.Regex != nil { + measurements := sg.MeasurementsByRegex(m.Regex.Val) + inputs := make([]query.Iterator, 0, len(measurements)) + if err := func() error { + for _, measurement := range measurements { + mm := m.Clone() + mm.Name = measurement + input, err := sg.CreateIterator(localCtx, mm, opt) + if err != nil { + return err + } + inputs = append(inputs, input) + } + return nil + }(); err != nil { + query.Iterators(inputs).Close() + return nil, err + } + + return query.Iterators(inputs).Merge(opt) + } + return sg.CreateIterator(localCtx, m, opt) +} + func (me *ClusterExecutor) CreateIterator(ctx context.Context, m *influxql.Measurement, opt query.IteratorOptions, shards []meta.ShardInfo) (query.Iterator, error) { type Result struct { iter query.Iterator err error } - n2s := NewNode2ShardIDs(me.MetaClient, me.Node, shards) - results := make(map[uint64]*Result) - - var mutex sync.Mutex - var wg sync.WaitGroup - fn := func(nodeId uint64, shardIDs []uint64) { - wg.Add(1) - go func() { - defer wg.Add(-1) + n2s := PlanNodes(me.Node.ID, shards, nil) - var iter query.Iterator - var err error - if nodeId == me.Node.ID { - //localCtx only use for local node - localCtx := ctx - iter, err = func() (query.Iterator, error) { - span := tracing.SpanFromContext(localCtx) - if span != nil { - span = span.StartSpan(fmt.Sprintf("local_node_id: %d", me.Node.ID)) - defer span.Finish() - - localCtx = tracing.NewContextWithSpan(localCtx, span) - } - sg := me.TSDBStore.ShardGroup(shardIDs) - if m.Regex != nil { - measurements := sg.MeasurementsByRegex(m.Regex.Val) - inputs := make([]query.Iterator, 0, len(measurements)) - if err := func() error { - for _, measurement := range measurements { - mm := m.Clone() - mm.Name = measurement - input, err := sg.CreateIterator(localCtx, mm, opt) - if err != nil { - return err - } - inputs = append(inputs, input) - } - return nil - }(); err != nil { - query.Iterators(inputs).Close() - return nil, err - } - - return query.Iterators(inputs).Merge(opt) - } - return sg.CreateIterator(localCtx, m, opt) - }() - } else { - iter, err = me.RemoteNodeExecutor.CreateIterator(nodeId, ctx, m, opt, shardIDs) - } + fn := func(nodeId uint64, shards []meta.ShardInfo) (result interface{}, err error) { + var iter query.Iterator + shardIDs := toShardIDs(shards) + if nodeId == me.Node.ID { + //localCtx only use for local node + iter, err = me.createLocalIteratorfunc(ctx, m, shardIDs, opt) + } else { + iter, err = me.RemoteNodeExecutor.CreateIterator(nodeId, ctx, m, opt, shardIDs) + } - mutex.Lock() - results[nodeId] = &Result{iter: iter, err: err} - mutex.Unlock() - return - }() + result = &Result{iter: iter, err: err} + return } - - n2s.Apply(fn) - wg.Wait() + result, _ := n2s.ExecuteWithRetry(fn) seriesN := 0 - inputs := make([]query.Iterator, 0, len(results)) - for _, r := range results { + var errOccur error + inputs := make([]query.Iterator, 0, len(result)) + for _, t := range result { + r, ok := t.(*Result) + if !ok { + continue + } if r.err != nil { - return nil, r.err + errOccur = r.err + break } if r.iter != nil { stats := r.iter.Stats() @@ -442,6 +429,19 @@ func (me *ClusterExecutor) CreateIterator(ctx context.Context, m *influxql.Measu inputs = append(inputs, r.iter) } } + if errOccur != nil { + // close all iterators + for _, t := range result { + r, ok := t.(*Result) + if !ok { + continue + } + if r.iter != nil { + r.iter.Close() + } + } + return nil, errOccur + } if opt.MaxSeriesN > 0 && seriesN > opt.MaxSeriesN { query.Iterators(inputs).Close() @@ -452,13 +452,14 @@ func (me *ClusterExecutor) CreateIterator(ctx context.Context, m *influxql.Measu func (me *ClusterExecutor) MapType(m *influxql.Measurement, field string, shards []meta.ShardInfo) influxql.DataType { type Result struct { + nodeId uint64 dataType influxql.DataType err error } - n2s := NewNode2ShardIDs(me.MetaClient, me.Node, shards) - var result Result - fn := func(nodeId uint64, shardIDs []uint64) { + n2s := PlanNodes(me.Node.ID, shards, nil) + fn := func(nodeId uint64, shards []meta.ShardInfo) (result interface{}, err error) { + shardIDs := toShardIDs(shards) if nodeId == me.Node.ID { sg := me.TSDBStore.ShardGroup(shardIDs) var names []string @@ -478,42 +479,43 @@ func (me *ClusterExecutor) MapType(m *influxql.Measurement, field string, shards typ = t } } - result.dataType = typ + result = &Result{dataType: typ, err: nil, nodeId: nodeId} } return } - - n2s.Apply(fn) - if result.dataType != influxql.Unknown { - return result.dataType + result, _ := n2s.ExecuteWithRetry(fn) + for _, t := range result { + if t == nil { + continue + } + r, ok := t.(*Result) + if !ok { + continue + } + if r.dataType != influxql.Unknown { + return r.dataType + } } - //本地失败, 尝试从remote node获取 - results := make(map[uint64]*Result) - var mutex sync.Mutex - var wg sync.WaitGroup - fn = func(nodeId uint64, shardIDs []uint64) { - wg.Add(1) - go func() { - defer wg.Add(-1) - - if nodeId != me.Node.ID { - mutex.Lock() - typ, err := me.RemoteNodeExecutor.MapType(nodeId, m, field, shardIDs) - results[nodeId] = &Result{dataType: typ, err: err} - mutex.Unlock() - } - return - }() + //本地失败, 尝试仅从remote node获取 + fn = func(nodeId uint64, shards []meta.ShardInfo) (result interface{}, err error) { + shardIDs := toShardIDs(shards) + if nodeId != me.Node.ID { + typ, err := me.RemoteNodeExecutor.MapType(nodeId, m, field, shardIDs) + result = &Result{dataType: typ, err: err} + } + return } - - n2s.Apply(fn) - wg.Wait() + result, _ = n2s.ExecuteWithRetry(fn) typ := influxql.Unknown - for nodeId, r := range results { + for _, t := range result { + r, ok := t.(*Result) + if !ok { + continue + } if r.err != nil { - me.Logger.Warn("results have error", zap.Error(r.err), zap.Uint64("node", nodeId)) + me.Logger.Warn("results have error", zap.Error(r.err), zap.Uint64("node", r.nodeId)) continue } if typ.LessThan(r.dataType) { @@ -530,52 +532,44 @@ func (me *ClusterExecutor) IteratorCost(m *influxql.Measurement, opt query.Itera err error } - n2s := NewNode2ShardIDs(me.MetaClient, me.Node, shards) - results := make(map[uint64]*Result) - - var mutex sync.Mutex - var wg sync.WaitGroup - fn := func(nodeId uint64, shardIDs []uint64) { - wg.Add(1) - go func() { - defer wg.Add(-1) + n2s := PlanNodes(me.Node.ID, shards, nil) - var cost query.IteratorCost - var err error - if nodeId == me.Node.ID { - sg := me.TSDBStore.ShardGroup(shardIDs) - if m.Regex != nil { - cost, err = func() (query.IteratorCost, error) { - var costs query.IteratorCost - measurements := sg.MeasurementsByRegex(m.Regex.Val) - for _, measurement := range measurements { - c, err := sg.IteratorCost(measurement, opt) - if err != nil { - return c, err - } - costs = costs.Combine(c) + fn := func(nodeId uint64, shards []meta.ShardInfo) (result interface{}, err error) { + shardIDs := toShardIDs(shards) + var cost query.IteratorCost + if nodeId == me.Node.ID { + sg := me.TSDBStore.ShardGroup(shardIDs) + if m.Regex != nil { + cost, err = func() (query.IteratorCost, error) { + var costs query.IteratorCost + measurements := sg.MeasurementsByRegex(m.Regex.Val) + for _, measurement := range measurements { + c, err := sg.IteratorCost(measurement, opt) + if err != nil { + return c, err } - return costs, nil - }() - } else { - cost, err = sg.IteratorCost(m.Name, opt) - } + costs = costs.Combine(c) + } + return costs, nil + }() } else { - cost, err = me.RemoteNodeExecutor.IteratorCost(nodeId, m, opt, shardIDs) + cost, err = sg.IteratorCost(m.Name, opt) } + } else { + cost, err = me.RemoteNodeExecutor.IteratorCost(nodeId, m, opt, shardIDs) + } - mutex.Lock() - results[nodeId] = &Result{cost: cost, err: err} - mutex.Unlock() - return - }() + result = &Result{cost: cost, err: err} + return } - - n2s.Apply(fn) - wg.Wait() + result, _ := n2s.ExecuteWithRetry(fn) var costs query.IteratorCost - for _, r := range results { + for _, t := range result { + r, ok := t.(*Result) + if !ok { + continue + } if r.err != nil { return costs, r.err } @@ -591,45 +585,39 @@ func (me *ClusterExecutor) FieldDimensions(m *influxql.Measurement, shards []met err error } - n2s := NewNode2ShardIDs(me.MetaClient, me.Node, shards) - results := make(map[uint64]*Result) + n2s := PlanNodes(me.Node.ID, shards, nil) - var mutex sync.Mutex - var wg sync.WaitGroup - fn := func(nodeId uint64, shardIDs []uint64) { - wg.Add(1) - go func() { - defer wg.Add(-1) - - var fields map[string]influxql.DataType - var dimensions map[string]struct{} - var err error - if nodeId == me.Node.ID { - sg := me.TSDBStore.ShardGroup(shardIDs) - var measurements []string - if m.Regex != nil { - measurements = sg.MeasurementsByRegex(m.Regex.Val) - } else { - measurements = []string{m.Name} - } - fields, dimensions, err = sg.FieldDimensions(measurements) + fn := func(nodeId uint64, shards []meta.ShardInfo) (result interface{}, err error) { + var fields map[string]influxql.DataType + var dimensions map[string]struct{} + shardIDs := toShardIDs(shards) + if nodeId == me.Node.ID { + sg := me.TSDBStore.ShardGroup(shardIDs) + var measurements []string + if m.Regex != nil { + measurements = sg.MeasurementsByRegex(m.Regex.Val) } else { - fields, dimensions, err = me.RemoteNodeExecutor.FieldDimensions(nodeId, m, shardIDs) + measurements = []string{m.Name} } - - mutex.Lock() - results[nodeId] = &Result{fields: fields, dimensions: dimensions, err: err} - mutex.Unlock() - return - }() + fields, dimensions, err = sg.FieldDimensions(measurements) + } else { + fields, dimensions, err = me.RemoteNodeExecutor.FieldDimensions(nodeId, m, shardIDs) + } + if fields != nil { + result = &Result{fields: fields, dimensions: dimensions, err: err} + } + return } - n2s.Apply(fn) - wg.Wait() + result, err := n2s.ExecuteWithRetry(fn) fields = make(map[string]influxql.DataType) dimensions = make(map[string]struct{}) - for _, r := range results { + for _, t := range result { + r, ok := t.(*Result) + if !ok { + continue + } //TODO: merge err if r.err != nil { return nil, nil, r.err @@ -638,7 +626,7 @@ func (me *ClusterExecutor) FieldDimensions(m *influxql.Measurement, shards []met for f, t := range r.fields { fields[f] = t } - for d, _ := range r.dimensions { + for d := range r.dimensions { dimensions[d] = struct{}{} } } @@ -676,39 +664,34 @@ func (me *ClusterExecutor) TagKeys(auth query.Authorizer, ids []uint64, cond inf return nil, err } - n2s := NewNode2ShardIDs(me.MetaClient, me.Node, shards) - result := make(map[uint64]*TagKeysResult) - - var mutex sync.Mutex - var wg sync.WaitGroup - fn := func(nodeId uint64, shardIDs []uint64) { - wg.Add(1) - go func() { - defer wg.Add(-1) + n2s := PlanNodes(me.Node.ID, shards, nil) - var tagKeys []tsdb.TagKeys - var err error - if nodeId == me.Node.ID { - tagKeys, err = me.TSDBStore.TagKeys(auth, shardIDs, cond) - } else { - tagKeys, err = me.RemoteNodeExecutor.TagKeys(nodeId, shardIDs, cond) - } + fn := func(nodeId uint64, shards []meta.ShardInfo) (result interface{}, err error) { + var tagKeys []tsdb.TagKeys + shardIDs := toShardIDs(shards) + if nodeId == me.Node.ID { + tagKeys, err = me.TSDBStore.TagKeys(auth, shardIDs, cond) + } else { + tagKeys, err = me.RemoteNodeExecutor.TagKeys(nodeId, shardIDs, cond) + } - if err != nil { - me.Logger.Error("TagKeys fail", zap.Error(err), zap.Uint64("node", nodeId)) - } - mutex.Lock() - result[nodeId] = &TagKeysResult{keys: tagKeys, err: err} - mutex.Unlock() - return - }() + if err != nil { + me.Logger.Error("TagKeys fail", zap.Error(err), zap.Uint64("node", nodeId)) + } + if len(tagKeys) > 0 { + result = &TagKeysResult{keys: tagKeys, err: err} + } + return } - n2s.Apply(fn) - wg.Wait() + result, err := n2s.ExecuteWithRetry(fn) uniqKeys := make(map[string]map[string]struct{}) - for _, r := range result { + for _, t := range result { + r, ok := t.(*TagKeysResult) + if !ok { + continue + } if r.err != nil { return nil, r.err } @@ -730,7 +713,7 @@ func (me *ClusterExecutor) TagKeys(auth query.Authorizer, ids []uint64, cond inf for m, keys := range uniqKeys { tagKey := &tagKeys[idx] tagKey.Measurement = m - for k, _ := range keys { + for k := range keys { tagKey.Keys = append(tagKey.Keys, k) } sort.Sort(StringSlice(tagKey.Keys)) @@ -750,31 +733,28 @@ func (me *ClusterExecutor) DeleteMeasurement(database, name string) error { if err != nil { return err } - nodes := NewNodeIdsByShards(shards) + nodes := getAllRelatedNodes(shards) results := make(map[uint64]*Result) var mutex sync.Mutex var wg sync.WaitGroup - fn := func(nodeId uint64) { + for _, nodeId := range nodes { wg.Add(1) - go func() { - defer wg.Add(-1) + go func(curNodeId uint64) { + defer wg.Done() var err error - if nodeId == me.Node.ID { + if curNodeId == me.Node.ID { err = me.TSDBStore.DeleteMeasurement(database, name) } else { - err = me.RemoteNodeExecutor.DeleteMeasurement(nodeId, database, name) + err = me.RemoteNodeExecutor.DeleteMeasurement(curNodeId, database, name) } mutex.Lock() - results[nodeId] = &Result{err: err} - mutex.Unlock() - return - }() + defer mutex.Unlock() + results[curNodeId] = &Result{err: err} + }(nodeId) } - - nodes.Apply(fn) wg.Wait() for _, r := range results { @@ -794,31 +774,29 @@ func (me *ClusterExecutor) DeleteDatabase(database string) error { if err != nil { return err } - nodes := NewNodeIdsByShards(shards) - results := make(map[uint64]*Result) + nodes := getAllRelatedNodes(shards) + results := make(map[uint64]*Result, len(nodes)) var mutex sync.Mutex var wg sync.WaitGroup - fn := func(nodeId uint64) { + for _, nodeId := range nodes { wg.Add(1) - go func() { - defer wg.Add(-1) + go func(curNodeId uint64) { + defer wg.Done() var err error - if nodeId == me.Node.ID { + if curNodeId == me.Node.ID { err = me.TSDBStore.DeleteDatabase(database) } else { - err = me.RemoteNodeExecutor.DeleteDatabase(nodeId, database) + err = me.RemoteNodeExecutor.DeleteDatabase(curNodeId, database) } mutex.Lock() - results[nodeId] = &Result{err: err} - mutex.Unlock() + defer mutex.Unlock() + results[curNodeId] = &Result{err: err} return - }() + }(nodeId) } - - nodes.Apply(fn) wg.Wait() for _, r := range results { @@ -838,33 +816,30 @@ func (me *ClusterExecutor) DeleteSeries(database string, sources []influxql.Sour if err != nil { return err } - nodes := NewNodeIdsByShards(shards) + nodes := getAllRelatedNodes(shards) results := make(map[uint64]*Result) var mutex sync.Mutex var wg sync.WaitGroup - fn := func(nodeId uint64) { + for _, nodeId := range nodes { wg.Add(1) - go func() { - defer wg.Add(-1) + go func(curNodeId uint64) { + defer wg.Done() var err error - if nodeId == me.Node.ID { + if curNodeId == me.Node.ID { // Convert "now()" to current time. cond = influxql.Reduce(cond, &influxql.NowValuer{Now: time.Now().UTC()}) err = me.TSDBStore.DeleteSeries(database, sources, cond) } else { - err = me.RemoteNodeExecutor.DeleteSeries(nodeId, database, sources, cond) + err = me.RemoteNodeExecutor.DeleteSeries(curNodeId, database, sources, cond) } mutex.Lock() - results[nodeId] = &Result{err: err} - mutex.Unlock() - return - }() + defer mutex.Unlock() + results[curNodeId] = &Result{err: err} + }(nodeId) } - - nodes.Apply(fn) wg.Wait() for _, r := range results { @@ -881,121 +856,34 @@ func (a StringSlice) Len() int { return len(a) } func (a StringSlice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a StringSlice) Less(i, j int) bool { return a[i] < a[j] } -type NodeIds []uint64 - -func NewNodeIdsByNodes(nodeInfos []meta.NodeInfo) NodeIds { - var ids []uint64 - for _, ni := range nodeInfos { - ids = append(ids, ni.ID) +func toNodeIds(nodeInfos []meta.NodeInfo) []uint64 { + ids := make([]uint64, len(nodeInfos)) + for i, ni := range nodeInfos { + ids[i] = ni.ID } - return NodeIds(ids) + return ids } -//TODO:取个达意的名字 -func NewNodeIdsByShards(Shards []meta.ShardInfo) NodeIds { - m := make(map[uint64]struct{}) - for _, si := range Shards { - for _, owner := range si.Owners { - m[owner.NodeID] = struct{}{} +// getAllRelatedNodes returns all related nodes owning shards +func getAllRelatedNodes(shards []meta.ShardInfo) []uint64 { + nodeMap := make(map[uint64]bool) + for _, shard := range shards { + for _, n := range shard.Owners { + nodeMap[n.NodeID] = true } } - nodes := make([]uint64, 0, len(m)) - for n, _ := range m { - nodes = append(nodes, n) + ids := make([]uint64, 0, len(nodeMap)) + for id := range nodeMap { + ids = append(ids, id) } - return nodes + return ids } -func (me NodeIds) Apply(fn func(nodeId uint64)) { - for _, nodeID := range me { - fn(nodeID) - } -} - -type Node2ShardIDs map[uint64][]uint64 - -func NewNode2ShardIDs(mc interface { - DataNode(nodeId uint64) (*meta.NodeInfo, error) -}, - localNode *influxdb.Node, - shards []meta.ShardInfo) Node2ShardIDs { - allNodes := make([]uint64, 0) - for _, si := range shards { - if si.OwnedBy(localNode.ID) { - continue - } - - for _, owner := range si.Owners { - allNodes = append(allNodes, owner.NodeID) - } - } - - //选出 active node - activeNodes := make(map[uint64]struct{}) - var wg sync.WaitGroup - var mutex sync.Mutex - for _, id := range allNodes { - wg.Add(1) - go func(id uint64) { - defer wg.Add(-1) - dialer := &NodeDialer{ - MetaClient: mc, - Timeout: 100 * time.Millisecond, //TODO: from config - } - - conn, err := dialer.DialNode(id) - if err != nil { - return - } - defer conn.Close() - mutex.Lock() - activeNodes[id] = struct{}{} - mutex.Unlock() - }(id) - } - - wg.Wait() - - shardIDsByNodeID := make(map[uint64][]uint64) - for _, si := range shards { - var nodeID uint64 - if si.OwnedBy(localNode.ID) { - nodeID = localNode.ID - } else if len(si.Owners) > 0 { - nodeID = si.Owners[rand.Intn(len(si.Owners))].NodeID - if _, ok := activeNodes[nodeID]; !ok { - //利用map的顺序不确定特性,随机选一个active的owner - randomOwners := make(map[uint64]struct{}) - for _, owner := range si.Owners { - randomOwners[owner.NodeID] = struct{}{} - } - for id, _ := range randomOwners { - if _, ok := activeNodes[id]; ok { - nodeID = id - break - } - } - } - } else { - continue - } - shardIDsByNodeID[nodeID] = append(shardIDsByNodeID[nodeID], si.ID) - } - - return shardIDsByNodeID -} - -type uint64Slice []uint64 - -func (a uint64Slice) Len() int { return len(a) } -func (a uint64Slice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a uint64Slice) Less(i, j int) bool { return a[i] < a[j] } - -func (me Node2ShardIDs) Apply(fn func(nodeId uint64, shardIDs []uint64)) { - for nodeID, shardIDs := range me { - // Sort shard IDs so we get more predicable execution. - sort.Sort(uint64Slice(shardIDs)) - fn(nodeID, shardIDs) +func toShardIDs(shards []meta.ShardInfo) []uint64 { + ids := make([]uint64, len(shards)) + for i, shard := range shards { + ids[i] = shard.ID } + return ids } diff --git a/coordinator/cluster_meta_client.go b/coordinator/cluster_meta_client.go index ab15f22..37b2a44 100644 --- a/coordinator/cluster_meta_client.go +++ b/coordinator/cluster_meta_client.go @@ -2,10 +2,12 @@ package coordinator import ( "fmt" + "time" + "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxql" "go.uber.org/zap" - "time" + "golang.org/x/time/rate" imeta "github.com/angopher/chronus/services/meta" ) @@ -22,15 +24,18 @@ func NewMetaClient(mc *meta.Config, cc Config, nodeID uint64) *ClusterMetaClient return &ClusterMetaClient{ NodeID: nodeID, metaCli: &MetaClientImpl{ - Addrs: cc.MetaServices, + Addrs: cc.MetaServices, + Logger: zap.NewNop(), }, pingIntervalMs: cc.PingMetaServiceIntervalMs, cache: imeta.NewClient(mc), + Logger: zap.NewNop(), } } func (me *ClusterMetaClient) WithLogger(log *zap.Logger) { me.Logger = log.With(zap.String("coordinator", "ClusterMetaClient")) + me.metaCli.Logger = log.With(zap.String("coordinator", "MetaClientImpl")) } func (me *ClusterMetaClient) Open() error { @@ -55,58 +60,96 @@ func (me *ClusterMetaClient) ClusterID() uint64 { } func (me *ClusterMetaClient) syncData() error { - fmt.Println("start sync data") + me.Logger.Info("start sync data") data, err := me.metaCli.Data() if err != nil { - fmt.Println("start sync fail ", err) + me.Logger.Error(fmt.Sprintf("start sync fail: %v", err)) return err } - fmt.Println("start sync done") + me.Logger.Info("start sync done") + if len(data.MetaNodes) > 0 { + addrs := make([]string, len(data.MetaNodes)) + for i, metaNode := range data.MetaNodes { + addrs[i] = metaNode.Host + } + me.metaCli.UpdateAddrs(addrs) + } return me.cache.ReplaceData(data) } -func (me *ClusterMetaClient) RunSyncLoop() { - //sync data first and will signal changes - me.syncData() - go func() { - printTicker := time.NewTicker(10 * time.Second) - for { - //for print sync status - select { - case <-printTicker.C: - index, err := me.metaCli.Ping() - if err != nil { - me.Logger.Warn("Ping fail", zap.Error(err)) - continue - } - me.Logger.Info(fmt.Sprintf("index=%d local_index=%d", index, me.cache.Data().Index)) - } - } - }() - +func (me *ClusterMetaClient) syncLoop() { ticker := time.NewTicker(time.Duration(me.pingIntervalMs) * time.Millisecond) + printLimiter := rate.NewLimiter(0.1, 1) + pingFailed := 0 for { select { case <-ticker.C: + needSync := false index, err := me.metaCli.Ping() if err != nil { me.Logger.Warn("Ping fail", zap.Error(err)) - continue + pingFailed++ + if pingFailed > 30 { + needSync = true + pingFailed = 0 + } else { + continue + } + } else { + switch { + case index > me.cache.DataIndex(): + needSync = true + case index < me.cache.DataIndex(): + me.Logger.Warn(fmt.Sprintf("index:%d < local index:%d", index, me.cache.DataIndex())) + default: + // normal + if printLimiter.Allow() { + // one log in 10 seconds + me.Logger.Debug(fmt.Sprintf("index=%d local_index=%d", index, me.cache.Data().Index)) + } + } } - if index > me.cache.DataIndex() { + if needSync { if err := me.syncData(); err != nil { me.Logger.Warn("syncData fail", zap.Error(err)) } else { me.Logger.Info("syncData success") } - } else if index < me.cache.DataIndex() { - me.Logger.Warn(fmt.Sprintf("index:%d < local index:%d", index, me.cache.DataIndex())) } } } - return +} + +func (me *ClusterMetaClient) Start() { + // sync first synchronously + retryTimes := 0 +LOOP: + for { + retryTimes++ + errC := make(chan error, 1) + go func(c chan error) { + c <- me.syncData() + close(c) + }(errC) + + //wait sync meta data from meta server + err := <-errC + if err != nil { + // retry + if retryTimes < 10 { + me.Logger.Warn("Load data on startup failed, retry") + continue LOOP + } else { + panic("sync meta data failed") + } + } + + break LOOP + + } + go me.syncLoop() } func (me *ClusterMetaClient) CreateDatabase(name string) (*meta.DatabaseInfo, error) { @@ -122,6 +165,25 @@ func (me *ClusterMetaClient) DeleteDataNode(id uint64) error { } return me.cache.DeleteDataNode(id) } + +func (me *ClusterMetaClient) IsDataNodeFreezed(id uint64) bool { + return me.cache.IsDataNodeFreezed(id) +} + +func (me *ClusterMetaClient) FreezeDataNode(id uint64) error { + if err := me.metaCli.FreezeDataNode(id); err != nil { + return err + } + return me.cache.FreezeDataNode(id) +} + +func (me *ClusterMetaClient) UnfreezeDataNode(id uint64) error { + if err := me.metaCli.UnfreezeDataNode(id); err != nil { + return err + } + return me.cache.UnfreezeDataNode(id) +} + func (me *ClusterMetaClient) Database(name string) *meta.DatabaseInfo { return me.cache.Database(name) } @@ -159,12 +221,13 @@ func (me *ClusterMetaClient) CreateDataNode(httpAddr, tcpAddr string) (*meta.Nod return me.cache.CreateDataNode(httpAddr, tcpAddr) } +// DataNode returns the node information according to the id, if it's not existed a meta.ErrNodeNotFound is returned. func (me *ClusterMetaClient) DataNode(id uint64) (*meta.NodeInfo, error) { return me.cache.DataNode(id) } func (me *ClusterMetaClient) DataNodes() ([]meta.NodeInfo, error) { - return me.cache.DataNodes() + return me.cache.DataNodes(), nil } func (me *ClusterMetaClient) ShardOwner(id uint64) (string, string, *meta.ShardGroupInfo) { @@ -199,11 +262,9 @@ func (me *ClusterMetaClient) CreateSubscription(database, rp, name, mode string, return me.cache.CreateSubscription(database, rp, name, mode, destinations) } -func (me *ClusterMetaClient) CreateUser(name, password string, admin bool) (meta.User, error) { - if u, err := me.metaCli.CreateUser(name, password, admin); err != nil { - return u, err - } - return me.cache.CreateUser(name, password, admin) +func (me *ClusterMetaClient) CreateUser(name, password string, admin bool) (u meta.User, err error) { + u, err = me.metaCli.CreateUser(name, password, admin) + return } func (me *ClusterMetaClient) Databases() []meta.DatabaseInfo { @@ -285,14 +346,15 @@ func (me *ClusterMetaClient) DeleteShardGroup(database, policy string, id uint64 if err := me.metaCli.DeleteShardGroup(database, policy, id); err != nil { return err } - return me.cache.DeleteShardGroup(database, policy, id) + // To cache data DeletedAt is not important + return me.cache.DeleteShardGroup(database, policy, id, time.Now()) } func (me *ClusterMetaClient) PruneShardGroups() error { if err := me.metaCli.PruneShardGroups(); err != nil { return err } - return me.cache.PruneShardGroups() + return nil } func (me *ClusterMetaClient) PrecreateShardGroups(from, to time.Time) error { @@ -313,7 +375,7 @@ func (me *ClusterMetaClient) UpdateUser(name, password string) error { if err := me.metaCli.UpdateUser(name, password); err != nil { return err } - return me.cache.UpdateUser(name, password) + return nil } func (me *ClusterMetaClient) UserPrivilege(username, database string) (*influxql.Privilege, error) { diff --git a/coordinator/cluster_planning.go b/coordinator/cluster_planning.go new file mode 100644 index 0000000..92e332e --- /dev/null +++ b/coordinator/cluster_planning.go @@ -0,0 +1,183 @@ +package coordinator + +import ( + "errors" + "fmt" + "math/rand" + "os" + "sync" + + "github.com/angopher/chronus/x" + "github.com/influxdata/influxdb/services/meta" +) + +var ( + ErrRetry = errors.New("operation needs another chance") +) + +// QueryFn returns ErrRetry indicating operation needs one more try on another node +type QueryFn func(nodeId uint64, shardIds []meta.ShardInfo) (interface{}, error) + +// getOwners returns the availible owners filtered by blacklist +func getOwners(blacklist map[uint64]bool, owners []meta.ShardOwner) []meta.ShardOwner { + if len(blacklist) < 1 { + return owners + } + arr := make([]meta.ShardOwner, 0, len(owners)) + for _, owner := range owners { + if _, ok := blacklist[owner.NodeID]; ok { + continue + } + arr = append(arr, owner) + } + return arr +} + +// PlanNodes distributes shards to correct nodes including those local node doesn't have +// Remote planning is randomized. +// blacklist is used during retries after queries fail on nodes +func PlanNodes(currentNodeId uint64, shard_queried []meta.ShardInfo, blacklist map[uint64]bool) Node2ShardIDs { + // shuffle the shards for randomized node selecting + shards := make([]meta.ShardInfo, len(shard_queried)) + copy(shards, shard_queried) + rand.Shuffle(len(shards), func(i, j int) { + shards[i], shards[j] = shards[j], shards[i] + }) + shardsMap := make(map[uint64][]meta.ShardInfo, x.Min(len(shards), 8)) +LOOP: + for _, shard := range shards { + // local shard first + if shard.OwnedBy(currentNodeId) { + shardsMap[currentNodeId] = append(shardsMap[currentNodeId], shard) + continue LOOP + } + // remote + owners := getOwners(blacklist, shard.Owners) + if len(owners) < 1 { + // no node availible, skip + continue LOOP + } + // node has been selected before has higher priority + for _, owner := range owners { + if _, ok := shardsMap[owner.NodeID]; ok { + shardsMap[owner.NodeID] = append(shardsMap[owner.NodeID], shard) + continue LOOP + } + } + // random pick + selectedNodeId := owners[rand.Intn(len(owners))].NodeID + shardsMap[selectedNodeId] = append(shardsMap[selectedNodeId], shard) + } + + return shardsMap +} + +type Node2ShardIDs map[uint64][]meta.ShardInfo + +type uint64Slice []uint64 + +func (a uint64Slice) Len() int { return len(a) } +func (a uint64Slice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a uint64Slice) Less(i, j int) bool { return a[i] < a[j] } + +// ExecuteWithRetry executes the query with retry to guarantee all jobs could be executed successfully. +// errLast holds the last error captured but not means result should be empty if any error occurred +func (me Node2ShardIDs) ExecuteWithRetry(fn QueryFn) (result []interface{}, errLast error) { + var ( + lock sync.Mutex + wg sync.WaitGroup + ) + wg.Add(len(me)) + for nodeId := range me { + go func(n uint64, ss []meta.ShardInfo) { + defer wg.Done() + arr, err := executePlanWithRetry(n, ss, fn) + lock.Lock() + defer lock.Unlock() + if err != nil { + errLast = err + } + if len(arr) > 0 { + result = append(result, arr...) + } + }(nodeId, me[nodeId]) + } + wg.Wait() + return +} + +type planSchedule struct { + NodeID uint64 + Shards []meta.ShardInfo +} + +func executePlanWithRetry(nodeId uint64, shards []meta.ShardInfo, fn QueryFn) (result []interface{}, errLast error) { + var ( + skipNodes = make(map[uint64]bool) + arr []interface{} + shardsRemained []meta.ShardInfo + err error + ) + plans := []*planSchedule{ + {NodeID: nodeId, Shards: shards}, + } + for len(plans) > 0 { + arr, plans, err = executePlans(plans, fn) + if err != nil { + errLast = err + } + if len(arr) > 0 { + result = append(result, arr...) + } + if len(plans) > 0 { + // replan + shardsRemained = shardsRemained[:0] + for _, p := range plans { + skipNodes[p.NodeID] = true + shardsRemained = append(shardsRemained, p.Shards...) + } + replanned := PlanNodes(0, shardsRemained, skipNodes) + plans = plans[:0] + cnt := 0 + for n, ss := range replanned { + plans = append(plans, &planSchedule{n, ss}) + cnt += len(ss) + } + if cnt < len(shardsRemained) { + // not all shards can be planned + fmt.Fprintln(os.Stderr, "Not all shards can be replanned") + } + } + } + return +} + +func executePlans(plans []*planSchedule, fn QueryFn) (result []interface{}, need_retry []*planSchedule, err_last error) { + for _, plan := range plans { + obj, err := executePlanSingle(plan.NodeID, plan.Shards, fn) + if err == nil { + if obj != nil { + result = append(result, obj) + } + continue + } + fmt.Fprintln(os.Stderr, "execute remotely error:", err) + if err != ErrRetry { + err_last = err + continue + } + // need retry + need_retry = append(need_retry, plan) + } + return +} + +func executePlanSingle(nodeId uint64, shards []meta.ShardInfo, fn QueryFn) (interface{}, error) { + defer func() { + r := recover() + if r != nil { + fmt.Fprintln(os.Stderr, "panic recover on node", nodeId, ":", r) + } + }() + return fn(nodeId, shards) +} diff --git a/coordinator/cluster_planning_test.go b/coordinator/cluster_planning_test.go new file mode 100644 index 0000000..792d831 --- /dev/null +++ b/coordinator/cluster_planning_test.go @@ -0,0 +1,139 @@ +package coordinator + +import ( + "testing" + + "github.com/influxdata/influxdb/services/meta" + "github.com/stretchr/testify/assert" +) + +func TestGetOwners(t *testing.T) { + owners := []meta.ShardOwner{ + {NodeID: 1}, + {NodeID: 2}, + {NodeID: 3}, + {NodeID: 4}, + {NodeID: 5}, + {NodeID: 6}, + } + ori_len := len(owners) + + assert.Equal(t, ori_len, len(getOwners(nil, owners))) + assert.Equal(t, ori_len, len(getOwners(map[uint64]bool{}, owners))) + assert.Equal(t, ori_len, len(getOwners(map[uint64]bool{ + 0: true, + }, owners))) + assert.Equal(t, ori_len-1, len(getOwners(map[uint64]bool{ + 0: true, + 1: true, + }, owners))) + assert.Equal(t, ori_len-1, len(getOwners(map[uint64]bool{ + 0: true, + 4: true, + }, owners))) + assert.Equal(t, ori_len-2, len(getOwners(map[uint64]bool{ + 0: true, + 2: true, + 4: true, + }, owners))) + assert.Equal(t, 0, len(getOwners(map[uint64]bool{ + 0: true, + 1: true, + 2: true, + 3: true, + 4: true, + 5: true, + 6: true, + }, owners))) +} + +func verifyPlan(t *testing.T, plan Node2ShardIDs, shards []meta.ShardInfo) bool { + shardMap := make(map[uint64]meta.ShardInfo, len(shards)) + for _, shard := range shards { + shardMap[shard.ID] = shard + } + for nodeID, ss := range plan { + for _, s := range ss { + shard, ok := shardMap[s.ID] + assert.True(t, ok) + if !ok { + return false + } + assert.True(t, shard.OwnedBy(nodeID)) + if !shard.OwnedBy(nodeID) { + return false + } + delete(shardMap, s.ID) + } + } + assert.Empty(t, shardMap) + return true +} + +func TestPlanNodes(t *testing.T) { + shards := []meta.ShardInfo{ + { + ID: 1, + Owners: []meta.ShardOwner{ + {NodeID: 1}, + {NodeID: 3}, + }, + }, { + ID: 2, + Owners: []meta.ShardOwner{ + {NodeID: 2}, + {NodeID: 4}, + }, + }, { + ID: 3, + Owners: []meta.ShardOwner{ + {NodeID: 1}, + {NodeID: 5}, + }, + }, { + ID: 4, + Owners: []meta.ShardOwner{ + {NodeID: 2}, + {NodeID: 5}, + }, + }, + } + + result := PlanNodes(0, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + result = PlanNodes(0, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + result = PlanNodes(0, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + result = PlanNodes(0, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + result = PlanNodes(0, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + result = PlanNodes(0, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + + // local node + resultLocal := PlanNodes(1, shards, nil) + assert.True(t, verifyPlan(t, resultLocal, shards)) + result = PlanNodes(1, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + assert.Equal(t, len(resultLocal[1]), len(result[1])) + result = PlanNodes(1, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + assert.Equal(t, len(resultLocal[1]), len(result[1])) + result = PlanNodes(1, shards, nil) + assert.True(t, verifyPlan(t, result, shards)) + assert.Equal(t, len(resultLocal[1]), len(result[1])) + + // black list + result = PlanNodes(0, shards, map[uint64]bool{ + 1: true, + }) + assert.True(t, verifyPlan(t, result, shards)) + assert.Equal(t, 0, len(result[1])) + result = PlanNodes(0, shards, map[uint64]bool{ + 2: true, + }) + assert.True(t, verifyPlan(t, result, shards)) + assert.Equal(t, 0, len(result[2])) +} diff --git a/coordinator/config.go b/coordinator/config.go index c9bfbd0..43bd0fb 100644 --- a/coordinator/config.go +++ b/coordinator/config.go @@ -23,8 +23,8 @@ const ( // A value of zero will make the maximum query limit unlimited. DefaultMaxConcurrentQueries = 0 - DefaultPoolMaxIdleStreams = 100 - DefaultPoolMaxConnections = 200 + DefaultPoolMinStreamsPerNode = 5 + DefaultPoolMaxStreamsPerNode = 200 // DefaultMaxSelectPointN is the maximum number of points a SELECT can process. // A value of zero will make the maximum point count unlimited. @@ -41,8 +41,8 @@ const ( type Config struct { DailTimeout toml.Duration `toml:"dial-timeout"` PoolMaxIdleTimeout toml.Duration `toml:"pool-max-idle-time"` - PoolMaxIdleStreams int `toml:"pool-max-idle-streams"` - PoolMaxConnections int `toml:"pool-max-connections"` + PoolMinStreamsPerNode int `toml:"pool-min-streams-per-node"` + PoolMaxStreamsPerNode int `toml:"pool-max-streams-per-node"` ShardReaderTimeout toml.Duration `toml:"shard-reader-timeout"` ClusterTracing bool `toml:"cluster-tracing"` WriteTimeout toml.Duration `toml:"write-timeout"` @@ -61,8 +61,8 @@ func NewConfig() Config { return Config{ DailTimeout: toml.Duration(DefaultDialTimeout), PoolMaxIdleTimeout: toml.Duration(DefaultPoolMaxIdleTimeout), - PoolMaxIdleStreams: DefaultPoolMaxIdleStreams, - PoolMaxConnections: DefaultPoolMaxConnections, + PoolMinStreamsPerNode: DefaultPoolMinStreamsPerNode, + PoolMaxStreamsPerNode: DefaultPoolMaxStreamsPerNode, ShardReaderTimeout: toml.Duration(DefaultShardReaderTimeout), ClusterTracing: false, WriteTimeout: toml.Duration(DefaultWriteTimeout), @@ -71,7 +71,7 @@ func NewConfig() Config { MaxSelectPointN: DefaultMaxSelectPointN, MaxSelectSeriesN: DefaultMaxSelectSeriesN, MetaServices: []string{DefaultMetaService}, - PingMetaServiceIntervalMs: 50, + PingMetaServiceIntervalMs: 250, } } @@ -80,8 +80,8 @@ func (c Config) Diagnostics() (*diagnostics.Diagnostics, error) { return diagnostics.RowFromMap(map[string]interface{}{ "dail-timeout": c.DailTimeout, "pool-max-idle-time": c.PoolMaxIdleTimeout, - "pool-max-idle-streams": c.PoolMaxIdleStreams, - "pool-max-connections": c.PoolMaxConnections, + "pool-min-streams-per-node": c.PoolMinStreamsPerNode, + "pool-max-streams-per-node": c.PoolMaxStreamsPerNode, "shard-reader-timeout": c.ShardReaderTimeout, "cluster-tracing": c.ClusterTracing, "write-timeout": c.WriteTimeout, diff --git a/coordinator/internal/service_statistics.go b/coordinator/internal/service_statistics.go new file mode 100644 index 0000000..f3fb3f6 --- /dev/null +++ b/coordinator/internal/service_statistics.go @@ -0,0 +1,104 @@ +package internal + +import ( + "sync/atomic" + + "github.com/influxdata/influxdb/models" +) + +// Statistics maintained by the cluster package +const ( + writeShardReq = "writeShardReq" + writeShardPointsReq = "writeShardPointsReq" + writeShardFail = "writeShardFail" + + createIteratorReq = "createIteratorReq" + createIteratorFail = "createIteratorFail" + + fieldDimensionsReq = "fieldDimensionsReq" + fieldDimensionsFail = "fieldDimensionsFail" + + tagKeysReq = "tagKeysReq" + tagKeysFail = "tagKeysFail" + + tagValuesReq = "tagValuesReq" + tagValuesFail = "tagValuesFail" + + measurementNamesReq = "measurementNamesReq" + measurementNamesFail = "measurementNamesFail" + + seriesCardinalityReq = "seriesCardinalityReq" + seriesCardinalityFail = "seriesCardinalityFail" + + iteratorCostReq = "iteratorCostReq" + iteratorCostFail = "iteratorCostFail" + + mapTypeReq = "mapTypeReq" + mapTypeFail = "mapTypeFail" +) + +type InternalServiceStatistics struct { + WriteShardReq int64 + WriteShardPointsReq int64 + WriteShardFail int64 + + CreateIteratorReq int64 + CreateIteratorFail int64 + + FieldDimensionsReq int64 + FieldDimensionsFail int64 + + TagKeysReq int64 + TagKeysFail int64 + + TagValuesReq int64 + TagValuesFail int64 + + MeasurementNamesReq int64 + MeasurementNamesFail int64 + + SeriesCardinalityReq int64 + SeriesCardinalityFail int64 + + IteratorCostReq int64 + IteratorCostFail int64 + + MapTypeReq int64 + MapTypeFail int64 +} + +func Statistics(stats *InternalServiceStatistics, tags map[string]string) []models.Statistic { + return []models.Statistic{{ + Name: "coordinator_service", + Tags: tags, + Values: map[string]interface{}{ + writeShardReq: atomic.LoadInt64(&stats.WriteShardReq), + writeShardPointsReq: atomic.LoadInt64(&stats.WriteShardPointsReq), + writeShardFail: atomic.LoadInt64(&stats.WriteShardFail), + + createIteratorReq: atomic.LoadInt64(&stats.CreateIteratorReq), + createIteratorFail: atomic.LoadInt64(&stats.CreateIteratorFail), + + fieldDimensionsReq: atomic.LoadInt64(&stats.FieldDimensionsReq), + fieldDimensionsFail: atomic.LoadInt64(&stats.FieldDimensionsFail), + + tagKeysReq: atomic.LoadInt64(&stats.TagKeysReq), + tagKeysFail: atomic.LoadInt64(&stats.TagKeysFail), + + tagValuesReq: atomic.LoadInt64(&stats.TagValuesReq), + tagValuesFail: atomic.LoadInt64(&stats.TagValuesFail), + + measurementNamesReq: atomic.LoadInt64(&stats.MeasurementNamesReq), + measurementNamesFail: atomic.LoadInt64(&stats.MeasurementNamesFail), + + seriesCardinalityReq: atomic.LoadInt64(&stats.SeriesCardinalityReq), + seriesCardinalityFail: atomic.LoadInt64(&stats.SeriesCardinalityFail), + + iteratorCostReq: atomic.LoadInt64(&stats.IteratorCostReq), + iteratorCostFail: atomic.LoadInt64(&stats.IteratorCostFail), + + mapTypeReq: atomic.LoadInt64(&stats.MapTypeReq), + mapTypeFail: atomic.LoadInt64(&stats.MapTypeFail), + }, + }} +} diff --git a/coordinator/meta_client_impl.go b/coordinator/meta_client_impl.go index afc978a..3266847 100644 --- a/coordinator/meta_client_impl.go +++ b/coordinator/meta_client_impl.go @@ -9,22 +9,55 @@ import ( "math/rand" "net" "net/http" + "os" + "sync" "time" "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxql" + "go.uber.org/zap" "github.com/angopher/chronus/raftmeta" imeta "github.com/angopher/chronus/services/meta" ) +var ( + client = http.Client{ + Transport: &http.Transport{ + Dial: func(netw, addr string) (net.Conn, error) { + c, err := net.DialTimeout(netw, addr, time.Second) + if err != nil { + return nil, err + } + return c, nil + }, + MaxConnsPerHost: 500, + MaxIdleConns: 100, + MaxIdleConnsPerHost: 20, + IdleConnTimeout: 60 * time.Second, + }, + } +) + type MetaClientImpl struct { - Addrs []string + Addrs []string + Logger *zap.Logger + + mu sync.Mutex } // return &MetaClientImpl{MetaServiceHost: "127.0.0.1:1234"} +func (me *MetaClientImpl) UpdateAddrs(addrs []string) { + me.mu.Lock() + defer me.mu.Unlock() + me.Addrs = addrs + me.Logger.Warn(fmt.Sprintf("The addresses of the meta servers have been updated: %v", addrs)) +} + func (me *MetaClientImpl) Url(path string) string { + me.mu.Lock() + defer me.mu.Unlock() return fmt.Sprintf("http://%s%s", me.Addrs[rand.Intn(len(me.Addrs))], path) } @@ -115,6 +148,27 @@ func (me *MetaClientImpl) DeleteDataNode(id uint64) error { return nil } +func (me *MetaClientImpl) freezeDataNode(id uint64, freeze bool) error { + req := raftmeta.FreezeDataNodeReq{Id: id, Freeze: freeze} + var resp raftmeta.FreezeDataNodeResp + err := RequestAndParseResponse(me.Url(raftmeta.FREEZE_DATA_NODE_PATH), &req, &resp) + if err != nil { + return err + } + + if resp.RetCode != 0 { + return errors.New(resp.RetMsg) + } + + return nil +} +func (me *MetaClientImpl) FreezeDataNode(id uint64) error { + return me.freezeDataNode(id, true) +} +func (me *MetaClientImpl) UnfreezeDataNode(id uint64) error { + return me.freezeDataNode(id, false) +} + func (me *MetaClientImpl) AddShardOwner(shardID, nodeID uint64) error { req := raftmeta.AddShardOwnerReq{ ShardID: shardID, @@ -579,21 +633,6 @@ func RequestAndParseResponse(url string, data interface{}, resp interface{}) err return err } req.Header.Set("Content-Type", "application/json") - req.Header.Set("Connection", "close") - - client := http.Client{ - Transport: &http.Transport{ - Dial: func(netw, addr string) (net.Conn, error) { - deadline := time.Now().Add(10 * time.Second) //TODO: timeout from config - c, err := net.DialTimeout(netw, addr, time.Second) - if err != nil { - return nil, err - } - c.SetDeadline(deadline) - return c, nil - }, - }, - } res, err := client.Do(req) if err != nil { @@ -606,5 +645,9 @@ func RequestAndParseResponse(url string, data interface{}, resp interface{}) err return err } - return json.Unmarshal(resBody, resp) + err = json.Unmarshal(resBody, resp) + if err != nil { + fmt.Fprintln(os.Stderr, "Error response(unmarshal failed):", string(resBody)) + } + return err } diff --git a/coordinator/points_writer.go b/coordinator/points_writer.go index a7ac455..521c214 100644 --- a/coordinator/points_writer.go +++ b/coordinator/points_writer.go @@ -405,8 +405,11 @@ func (w *PointsWriter) writeToShard(shard *meta.ShardInfo, database, retentionPo atomic.AddInt64(&w.stats.PointWriteReqRemote, int64(len(points))) err := w.ShardWriter.WriteShard(shardID, owner.NodeID, points) - if err != nil && IsRetryable(err) { - w.Logger.Warn("ShardWriter.WriteShard fail", zap.Error(err)) + if err != nil && canRetry(err) { + w.Logger.Warn(fmt.Sprintf( + "ShardWriter.WriteShard fail to %d and enqueue to hh", + owner.NodeID, + ), zap.Error(err)) // The remote write failed so queue it via hinted handoff atomic.AddInt64(&w.stats.WritePointReqHH, int64(len(points))) hherr := w.HintedHandoff.WriteShard(shardID, owner.NodeID, points) @@ -476,7 +479,7 @@ func (w *PointsWriter) writeToShard(shard *meta.ShardInfo, database, retentionPo return ErrWriteFailed } -func IsRetryable(err error) bool { +func canRetry(err error) bool { if err == nil { return true } diff --git a/coordinator/pool.go b/coordinator/pool.go deleted file mode 100644 index 8e15b00..0000000 --- a/coordinator/pool.go +++ /dev/null @@ -1,188 +0,0 @@ -package coordinator - -import ( - "errors" - "fmt" - "net" - "sync" - "sync/atomic" - "time" - - "gopkg.in/fatih/pool.v2" -) - -// boundedPool implements the Pool interface based on buffered channels. -type boundedPool struct { - // storage for our net.Conn connections - mu sync.Mutex - conns chan net.Conn - - timeout time.Duration - total int32 - // net.Conn generator - factory Factory -} - -// Factory is a function to create new connections. -type Factory func() (net.Conn, error) - -// NewBoundedPool returns a new pool based on buffered channels with an initial -// capacity, maximum capacity and timeout to wait for a connection from the pool. -// Factory is used when initial capacity is -// greater than zero to fill the pool. A zero initialCap doesn't fill the Pool -// until a new Get() is called. During a Get(), If there is no new connection -// available in the pool and total connections is less than the max, a new connection -// will be created via the Factory() method. Othewise, the call will block until -// a connection is available or the timeout is reached. -func NewBoundedPool(initialCap, maxCap int, timeout time.Duration, factory Factory) (pool.Pool, error) { - if initialCap < 0 || maxCap <= 0 || initialCap > maxCap { - return nil, errors.New("invalid capacity settings") - } - - c := &boundedPool{ - conns: make(chan net.Conn, maxCap), - factory: factory, - timeout: timeout, - } - - // create initial connections, if something goes wrong, - // just close the pool error out. - for i := 0; i < initialCap; i++ { - conn, err := factory() - if err != nil { - c.Close() - return nil, fmt.Errorf("factory is not able to fill the pool: %s", err) - } - c.conns <- conn - atomic.AddInt32(&c.total, 1) - } - - return c, nil -} - -func (c *boundedPool) getConns() chan net.Conn { - c.mu.Lock() - conns := c.conns - c.mu.Unlock() - return conns -} - -// Get implements the Pool interfaces Get() method. If there is no new -// connection available in the pool, a new connection will be created via the -// Factory() method. -func (c *boundedPool) Get() (net.Conn, error) { - conns := c.getConns() - if conns == nil { - return nil, pool.ErrClosed - } - - // Try and grab a connection from the pool - select { - case conn := <-conns: - if conn == nil { - return nil, pool.ErrClosed - } - return c.wrapConn(conn), nil - default: - // Could not get connection, can we create a new one? - if atomic.LoadInt32(&c.total) < int32(cap(conns)) { - conn, err := c.factory() - if err != nil { - return nil, err - } - atomic.AddInt32(&c.total, 1) - - return c.wrapConn(conn), nil - } - } - - // The pool was empty and we couldn't create a new one to - // retry until one is free or we timeout - select { - case conn := <-conns: - if conn == nil { - return nil, pool.ErrClosed - } - return c.wrapConn(conn), nil - case <-time.After(c.timeout): - return nil, fmt.Errorf("timed out waiting for free connection") - } - -} - -// put puts the connection back to the pool. If the pool is full or closed, -// conn is simply closed. A nil conn will be rejected. -func (c *boundedPool) put(conn net.Conn) error { - if conn == nil { - return errors.New("connection is nil. rejecting") - } - - c.mu.Lock() - defer c.mu.Unlock() - - if c.conns == nil { - // pool is closed, close passed connection - return conn.Close() - } - - // put the resource back into the pool. If the pool is full, this will - // block and the default case will be executed. - select { - case c.conns <- conn: - return nil - default: - // pool is full, close passed connection - return conn.Close() - } -} - -func (c *boundedPool) Close() { - c.mu.Lock() - conns := c.conns - c.conns = nil - c.factory = nil - c.mu.Unlock() - - if conns == nil { - return - } - - close(conns) - for conn := range conns { - conn.Close() - } -} - -func (c *boundedPool) Len() int { return len(c.getConns()) } - -// newConn wraps a standard net.Conn to a poolConn net.Conn. -func (c *boundedPool) wrapConn(conn net.Conn) net.Conn { - p := &pooledConn{c: c} - p.Conn = conn - return p -} - -// pooledConn is a wrapper around net.Conn to modify the the behavior of -// net.Conn's Close() method. -type pooledConn struct { - net.Conn - c *boundedPool - unusable bool -} - -// Close() puts the given connects back to the pool instead of closing it. -func (p pooledConn) Close() error { - if p.unusable { - if p.Conn != nil { - return p.Conn.Close() - } - return nil - } - return p.c.put(p.Conn) -} - -// MarkUnusable() marks the connection not usable any more, to let the pool close it instead of returning it to pool. -func (p *pooledConn) MarkUnusable() { - p.unusable = true - atomic.AddInt32(&p.c.total, -1) -} diff --git a/coordinator/remote_executor.go b/coordinator/remote_executor.go index 9ee5381..5b849f4 100644 --- a/coordinator/remote_executor.go +++ b/coordinator/remote_executor.go @@ -2,18 +2,23 @@ package coordinator import ( "context" + "encoding" "errors" + "io" "net" "time" + "github.com/angopher/chronus/x" "github.com/influxdata/influxdb/pkg/tracing" "github.com/influxdata/influxdb/query" - "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxdb/tsdb" "github.com/influxdata/influxql" + "go.uber.org/zap" ) type RemoteNodeExecutor interface { + WithLogger(log *zap.Logger) + TagKeys(nodeId uint64, ShardIDs []uint64, cond influxql.Expr) ([]tsdb.TagKeys, error) TagValues(nodeId uint64, shardIDs []uint64, cond influxql.Expr) ([]tsdb.TagValues, error) MeasurementNames(nodeId uint64, database string, cond influxql.Expr) ([][]byte, error) @@ -26,24 +31,57 @@ type RemoteNodeExecutor interface { MapType(nodeId uint64, m *influxql.Measurement, field string, shardIds []uint64) (influxql.DataType, error) CreateIterator(nodeId uint64, ctx context.Context, m *influxql.Measurement, opt query.IteratorOptions, shardIds []uint64) (query.Iterator, error) TaskManagerStatement(nodeId uint64, stmt influxql.Statement) (*query.Result, error) + Stats() []StatEntity } type remoteNodeExecutor struct { - MetaClient interface { - DataNode(nodeId uint64) (*meta.NodeInfo, error) - } + ClientPool *ClientPool DailTimeout time.Duration ShardReaderTimeout time.Duration ClusterTracing bool + Logger *zap.Logger } -func (me *remoteNodeExecutor) CreateIterator(nodeId uint64, ctx context.Context, m *influxql.Measurement, opt query.IteratorOptions, shardIds []uint64) (query.Iterator, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.ShardReaderTimeout, +func (executor *remoteNodeExecutor) WithLogger(log *zap.Logger) { + executor.Logger = log.With(zap.String("service", "RemoteNodeExecutor")) +} + +func writeTestPacket(conn net.Conn) (err error) { + conn.SetDeadline(time.Now().Add(500 * time.Millisecond)) + defer conn.SetDeadline(time.Time{}) + if _, err = conn.Write([]byte{testRequestMessage, 0, 0, 0, 0, 0, 0, 0, 0}); err != nil { + return + } + return +} + +func getConnWithRetry(pool *ClientPool, nodeId uint64, logger *zap.Logger) (x.PooledConn, error) { + var ( + conn x.PooledConn + err error + ) + retries := 3 + for retries > 0 { + retries-- + conn, err = pool.GetConn(nodeId) + if err != nil { + logger.Warn("Failed to get connection from pool", zap.Error(err)) + return nil, ErrRetry + } + // do write test + if err = writeTestPacket(conn); err != nil { + conn.MarkUnusable() + conn.Close() + logger.Warn("Failed to get connection from pool", zap.Error(err)) + continue + } + return conn, nil } + return nil, err +} - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) CreateIterator(nodeId uint64, ctx context.Context, m *influxql.Measurement, opt query.IteratorOptions, shardIds []uint64) (query.Iterator, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return nil, err } @@ -63,14 +101,16 @@ func (me *remoteNodeExecutor) CreateIterator(nodeId uint64, ctx context.Context, Opt: opt, SpanContex: spanCtx, }); err != nil { + conn.MarkUnusable() return err } // Read the response. - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != nil { - return err + return resp.Err } return nil @@ -79,23 +119,14 @@ func (me *remoteNodeExecutor) CreateIterator(nodeId uint64, ctx context.Context, return nil, err } - if resp.DataType == influxql.Unknown { - return nil, nil - } - stats := query.IteratorStats{SeriesN: resp.SeriesN} //conn.Close will be invoked when iterator.Close - itr := query.NewReaderIterator(ctx, conn, resp.DataType, stats) + itr := query.NewReaderIterator(ctx, newIteratorReader(conn, resp.Termination), resp.DataType, stats) return itr, nil } -func (me *remoteNodeExecutor) MapType(nodeId uint64, m *influxql.Measurement, field string, shardIds []uint64) (influxql.DataType, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) MapType(nodeId uint64, m *influxql.Measurement, field string, shardIds []uint64) (influxql.DataType, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return influxql.Unknown, err } @@ -113,10 +144,12 @@ func (me *remoteNodeExecutor) MapType(nodeId uint64, m *influxql.Measurement, fi Field: field, ShardIDs: shardIds, }); err != nil { + conn.MarkUnusable() return err } - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -130,13 +163,8 @@ func (me *remoteNodeExecutor) MapType(nodeId uint64, m *influxql.Measurement, fi return resp.DataType, nil } -func (me *remoteNodeExecutor) IteratorCost(nodeId uint64, m *influxql.Measurement, opt query.IteratorOptions, shardIds []uint64) (query.IteratorCost, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) IteratorCost(nodeId uint64, m *influxql.Measurement, opt query.IteratorOptions, shardIds []uint64) (query.IteratorCost, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return query.IteratorCost{}, err } @@ -149,10 +177,12 @@ func (me *remoteNodeExecutor) IteratorCost(nodeId uint64, m *influxql.Measuremen Opt: opt, ShardIDs: shardIds, }); err != nil { + conn.MarkUnusable() return err } - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -166,13 +196,8 @@ func (me *remoteNodeExecutor) IteratorCost(nodeId uint64, m *influxql.Measuremen return resp.Cost, nil } -func (me *remoteNodeExecutor) FieldDimensions(nodeId uint64, m *influxql.Measurement, shardIds []uint64) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) FieldDimensions(nodeId uint64, m *influxql.Measurement, shardIds []uint64) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return nil, nil, err } @@ -184,10 +209,12 @@ func (me *remoteNodeExecutor) FieldDimensions(nodeId uint64, m *influxql.Measure req.ShardIDs = shardIds req.Sources = influxql.Sources([]influxql.Source{m}) if err := EncodeTLV(conn, fieldDimensionsRequestMessage, &req); err != nil { + conn.MarkUnusable() return err } - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != nil { return resp.Err @@ -201,13 +228,8 @@ func (me *remoteNodeExecutor) FieldDimensions(nodeId uint64, m *influxql.Measure return resp.Fields, resp.Dimensions, nil } -func (me *remoteNodeExecutor) TaskManagerStatement(nodeId uint64, stmt influxql.Statement) (*query.Result, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) TaskManagerStatement(nodeId uint64, stmt influxql.Statement) (*query.Result, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return nil, err } @@ -219,10 +241,12 @@ func (me *remoteNodeExecutor) TaskManagerStatement(nodeId uint64, stmt influxql. req.SetStatement(stmt.String()) req.SetDatabase("") if err := EncodeTLV(conn, executeTaskManagerRequestMessage, &req); err != nil { + conn.MarkUnusable() return err } - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -238,13 +262,8 @@ func (me *remoteNodeExecutor) TaskManagerStatement(nodeId uint64, stmt influxql. return result, nil } -func (me *remoteNodeExecutor) SeriesCardinality(nodeId uint64, database string) (int64, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) SeriesCardinality(nodeId uint64, database string) (int64, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return -1, err } @@ -255,11 +274,13 @@ func (me *remoteNodeExecutor) SeriesCardinality(nodeId uint64, database string) if err := EncodeTLV(conn, seriesCardinalityRequestMessage, &SeriesCardinalityRequest{ Database: database, }); err != nil { + conn.MarkUnusable() return err } var resp SeriesCardinalityResponse - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -274,13 +295,8 @@ func (me *remoteNodeExecutor) SeriesCardinality(nodeId uint64, database string) return n, nil } -func (me *remoteNodeExecutor) MeasurementNames(nodeId uint64, database string, cond influxql.Expr) ([][]byte, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) MeasurementNames(nodeId uint64, database string, cond influxql.Expr) ([][]byte, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return nil, err } @@ -292,11 +308,13 @@ func (me *remoteNodeExecutor) MeasurementNames(nodeId uint64, database string, c Database: database, Cond: cond, }); err != nil { + conn.MarkUnusable() return err } var resp MeasurementNamesResponse - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -311,13 +329,8 @@ func (me *remoteNodeExecutor) MeasurementNames(nodeId uint64, database string, c return names, nil } -func (me *remoteNodeExecutor) TagValues(nodeId uint64, shardIDs []uint64, cond influxql.Expr) ([]tsdb.TagValues, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) TagValues(nodeId uint64, shardIDs []uint64, cond influxql.Expr) ([]tsdb.TagValues, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return nil, err } @@ -331,11 +344,13 @@ func (me *remoteNodeExecutor) TagValues(nodeId uint64, shardIDs []uint64, cond i Cond: cond, }, }); err != nil { + conn.MarkUnusable() return err } var resp TagValuesResponse - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -350,17 +365,11 @@ func (me *remoteNodeExecutor) TagValues(nodeId uint64, shardIDs []uint64, cond i return tagValues, nil } -func (me *remoteNodeExecutor) TagKeys(nodeId uint64, shardIDs []uint64, cond influxql.Expr) ([]tsdb.TagKeys, error) { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) TagKeys(nodeId uint64, shardIDs []uint64, cond influxql.Expr) ([]tsdb.TagKeys, error) { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return nil, err } - defer conn.Close() var tagKeys []tsdb.TagKeys @@ -369,11 +378,13 @@ func (me *remoteNodeExecutor) TagKeys(nodeId uint64, shardIDs []uint64, cond inf ShardIDs: shardIDs, Cond: cond, }); err != nil { + conn.MarkUnusable() return err } var resp TagKeysResponse - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -388,13 +399,8 @@ func (me *remoteNodeExecutor) TagKeys(nodeId uint64, shardIDs []uint64, cond inf return tagKeys, nil } -func (me *remoteNodeExecutor) DeleteMeasurement(nodeId uint64, database, name string) error { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) DeleteMeasurement(nodeId uint64, database, name string) error { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return err } @@ -405,11 +411,13 @@ func (me *remoteNodeExecutor) DeleteMeasurement(nodeId uint64, database, name st Database: database, Name: name, }); err != nil { + conn.MarkUnusable() return err } var resp DeleteMeasurementResponse - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -423,13 +431,8 @@ func (me *remoteNodeExecutor) DeleteMeasurement(nodeId uint64, database, name st return nil } -func (me *remoteNodeExecutor) DeleteDatabase(nodeId uint64, database string) error { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) DeleteDatabase(nodeId uint64, database string) error { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return err } @@ -439,11 +442,13 @@ func (me *remoteNodeExecutor) DeleteDatabase(nodeId uint64, database string) err if err := EncodeTLV(conn, deleteDatabaseRequestMessage, &DeleteDatabaseRequest{ Database: database, }); err != nil { + conn.MarkUnusable() return err } var resp DeleteDatabaseResponse - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -457,13 +462,8 @@ func (me *remoteNodeExecutor) DeleteDatabase(nodeId uint64, database string) err return nil } -func (me *remoteNodeExecutor) DeleteSeries(nodeId uint64, database string, sources []influxql.Source, cond influxql.Expr) error { - dialer := &NodeDialer{ - MetaClient: me.MetaClient, - Timeout: me.DailTimeout, - } - - conn, err := dialer.DialNode(nodeId) +func (executor *remoteNodeExecutor) DeleteSeries(nodeId uint64, database string, sources []influxql.Source, cond influxql.Expr) error { + conn, err := getConnWithRetry(executor.ClientPool, nodeId, executor.Logger) if err != nil { return err } @@ -475,11 +475,13 @@ func (me *remoteNodeExecutor) DeleteSeries(nodeId uint64, database string, sourc Sources: influxql.Sources(sources), Cond: cond, }); err != nil { + conn.MarkUnusable() return err } var resp DeleteSeriesResponse - if _, err := DecodeTLV(conn, &resp); err != nil { + if _, err := decodeTLV(conn, &resp); err != nil { + conn.MarkUnusable() return err } else if resp.Err != "" { return errors.New(resp.Err) @@ -493,32 +495,109 @@ func (me *remoteNodeExecutor) DeleteSeries(nodeId uint64, database string, sourc return nil } -// NodeDialer dials connections to a given node. -type NodeDialer struct { - MetaClient interface { - DataNode(nodeId uint64) (*meta.NodeInfo, error) +func (executor *remoteNodeExecutor) Stats() []StatEntity { + return executor.ClientPool.Stat() +} + +type iteratorReader struct { + io.ReadCloser + terminator []byte + buf []byte + reader io.ReadCloser + judger *x.CyclicBuffer + terminated bool + bytes int +} + +func newIteratorReader(rd io.ReadCloser, terminator []byte) *iteratorReader { + return &iteratorReader{ + reader: rd, + terminator: terminator, + buf: make([]byte, len(terminator)), + judger: x.NewCyclicBuffer(len(terminator)), } - Timeout time.Duration } -// DialNode returns a connection to a node. -func (d *NodeDialer) DialNode(nodeID uint64) (net.Conn, error) { - ni, err := d.MetaClient.DataNode(nodeID) - if err != nil { - return nil, err +func shiftBuffer(data []byte, header []byte) { + offset := len(header) + if offset > len(data) { + return + } + for i := len(data) - 1; i >= offset; i-- { + data[i] = data[i-offset] } + for i := 0; i < offset; i++ { + data[i] = header[i] + } +} + +func (r *iteratorReader) Read(p []byte) (n int, err error) { + if r.terminated { + return 0, io.EOF + } + n, err = r.reader.Read(p) + bytes := r.bytes + r.bytes += n + if n == 0 { + return 0, err + } + dumped := r.judger.Dump(r.buf) + r.judger.Write(p[:n]) + if r.bytes < len(r.terminator) { + return 0, nil + } + shiftBuffer(p, r.buf[:x.Min(dumped, n)]) + if r.judger.Compare(r.terminator) { + r.terminated = true + err = io.EOF + } + if bytes < len(r.buf) { + return n - (len(r.buf) - bytes), err + } + return n, err +} - conn, err := net.Dial("tcp", ni.TCPHost) +func (r *iteratorReader) consumeRest() (discarded int) { + if r.terminated { + return + } + buf := make([]byte, 32) + for { + n, err := r.Read(buf) + discarded += n + if err == io.EOF { + break + } + } + return +} + +func (r *iteratorReader) Close() error { + r.consumeRest() + return r.reader.Close() +} + +// decodeTLV reads the type-length-value record from r and unmarshal it into v. +func decodeTLV(conn net.Conn, v encoding.BinaryUnmarshaler) (typ byte, err error) { + typ, err = ReadType(conn) if err != nil { - return nil, err + return 0, err + } + if err := decodeLV(conn, v); err != nil { + return 0, err } - conn.SetDeadline(time.Now().Add(d.Timeout)) + return typ, nil +} - // Write the cluster multiplexing header byte - if _, err := conn.Write([]byte{MuxHeader}); err != nil { - conn.Close() - return nil, err +// decodeLV reads the length-value record from r and unmarshal it into v. +func decodeLV(conn net.Conn, v encoding.BinaryUnmarshaler) error { + buf, err := ReadLV(conn, 3*time.Second) + if err != nil { + return err } - return conn, nil + if err := v.UnmarshalBinary(buf); err != nil { + return err + } + return nil } diff --git a/coordinator/remote_executor_test.go b/coordinator/remote_executor_test.go index 01bc78f..326e0b9 100644 --- a/coordinator/remote_executor_test.go +++ b/coordinator/remote_executor_test.go @@ -1,9 +1,42 @@ -package coordinator_test +package coordinator import ( + "fmt" + "io" "testing" + + "github.com/stretchr/testify/assert" ) +// small buffer to test +func testWithBuffer(buf []byte) []byte { + rd, wr := io.Pipe() + defer rd.Close() + go func() { + defer wr.Close() + wr.Write([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}) + wr.Write([]byte{1, 2}) + fmt.Println("write end") + }() + var result []byte + reader := newIteratorReader(rd, []byte{9, 10, 11, 12, 13, 14, 15, 16}) + for { + n, err := reader.Read(buf) + if err != nil { + break + } + if n > 0 { + result = append(result, buf[:n]...) + } + } + return result +} + +func TestIteratorReader(t *testing.T) { + assert.Equal(t, []byte{1, 2, 3, 4, 5, 6, 7, 8}, testWithBuffer(make([]byte, 3))) + assert.Equal(t, []byte{1, 2, 3, 4, 5, 6, 7, 8}, testWithBuffer(make([]byte, 9))) +} + func TestRemoteNodeExecuterTagKeys(t *testing.T) { //TODO } diff --git a/coordinator/request/request.go b/coordinator/request/request.go new file mode 100644 index 0000000..4c337ab --- /dev/null +++ b/coordinator/request/request.go @@ -0,0 +1,93 @@ +package request + +import ( + "encoding/binary" + "fmt" + "io" + "sync" +) + +const ( + MAX_BODY = 1024 * 1024 * 1024 +) + +var ( + bufferPool = sync.Pool{ + New: func() interface{} { + return make([]byte, 64) + }, + } +) + +type ClusterMessage struct { + Type byte + Data []byte +} + +type ClusterMessageReader struct { + data []byte +} + +func (reader *ClusterMessageReader) fetch() *ClusterMessage { + sz := len(reader.data) + if sz < 9 { + return nil + } + l := binary.BigEndian.Uint64(reader.data[1:9]) + if l > MAX_BODY { + panic(fmt.Sprint("Error decoding cluster request, got a huge size of ", l)) + } + consumed := int(l + 8 + 1) + if sz < consumed { + // not enough + return nil + } + + data := make([]byte, l) + copy(data, reader.data[9:consumed]) + req := &ClusterMessage{ + Type: reader.data[0], + Data: data, + } + // shift the buffer + for i := 0; i < sz-consumed; i++ { + reader.data[i] = reader.data[i+consumed] + } + reader.data = reader.data[:sz-consumed] + return req +} + +func (reader *ClusterMessageReader) Read(r io.Reader) (*ClusterMessage, error) { + if req := reader.fetch(); req != nil { + return req, nil + } + buf := bufferPool.Get().([]byte) + defer bufferPool.Put(buf) + for { + n, err := r.Read(buf) + if n > 0 { + reader.data = append(reader.data, buf[:n]...) + if req := reader.fetch(); req != nil { + return req, nil + } + } + if err != nil { + return nil, err + } + } +} + +// Len returns the length of data in buffer +func (reader *ClusterMessageReader) Len() int { + return len(reader.data) +} + +// Data returns unprocessed data +func (reader *ClusterMessageReader) Data() []byte { + if len(reader.data) == 0 { + return []byte{} + } + data := make([]byte, len(reader.data)) + copy(data, reader.data) + return data +} diff --git a/coordinator/request/request_test.go b/coordinator/request/request_test.go new file mode 100644 index 0000000..11ccc0c --- /dev/null +++ b/coordinator/request/request_test.go @@ -0,0 +1,181 @@ +package request_test + +import ( + "errors" + "testing" + + "github.com/angopher/chronus/coordinator/request" + "github.com/stretchr/testify/assert" +) + +type readFn func(p []byte) (n int, err error) + +type mockReader struct { + reader readFn +} + +func (r *mockReader) Read(p []byte) (n int, err error) { + return r.reader(p) +} + +func TestRequestReaderTimeout(t *testing.T) { + reader := &request.ClusterMessageReader{} + req, err := reader.Read(&mockReader{ + reader: func(p []byte) (n int, err error) { + return 0, errors.New("timeout") + }, + }) + assert.Nil(t, req) + assert.NotNil(t, err) + assert.Contains(t, "timeout", err.Error()) +} + +func TestRequestReaderPartial(t *testing.T) { + step := 0 + reader := &request.ClusterMessageReader{} + req, err := reader.Read(&mockReader{ + reader: func(p []byte) (n int, err error) { + switch step { + case 0: + p[0] = 1 + p[1] = 0 + step++ + return 2, nil + case 1: + p[0] = 0 + p[1] = 0 + p[2] = 0 + p[3] = 0 + p[4] = 0 + step++ + return 5, nil + } + return 0, errors.New("timeout") + }, + }) + assert.Nil(t, req) + assert.NotNil(t, err) +} + +func TestRequestReaderNormal(t *testing.T) { + step := 0 + reader := &request.ClusterMessageReader{} + req, err := reader.Read(&mockReader{ + reader: func(p []byte) (n int, err error) { + switch step { + case 0: + p[0] = 1 + p[1] = 0 + step++ + return 2, nil + case 1: + p[0] = 0 + p[1] = 0 + p[2] = 0 + p[3] = 0 + p[4] = 0 + step++ + return 5, nil + case 2: + p[0] = 0 + p[1] = 2 + p[2] = 12 + step++ + return 3, nil + case 3: + p[0] = 34 + p[1] = 22 + step++ + return 2, errors.New("timeout") + } + return 0, errors.New("timeout") + }, + }) + assert.NotNil(t, req) + assert.Nil(t, err) + assert.Equal(t, byte(1), req.Type) + assert.Equal(t, []byte{12, 34}, req.Data) + assert.Equal(t, []byte{22}, reader.Data()) +} + +func TestRequestReaderMore(t *testing.T) { + step := 0 + reader := &request.ClusterMessageReader{} + r := &mockReader{ + reader: func(p []byte) (n int, err error) { + switch step { + case 0: + p[0] = 1 + p[1] = 0 + step++ + return 2, nil + case 1: + p[0] = 0 + p[1] = 0 + p[2] = 0 + p[3] = 0 + p[4] = 0 + step++ + return 5, nil + case 2: + p[0] = 0 + p[1] = 2 + p[2] = 12 + step++ + return 3, nil + case 3: + p[0] = 34 + p[1] = 22 + p[2] = 0 + step++ + return 3, errors.New("timeout") + case 4: + p[0] = 0 + p[1] = 0 + p[2] = 0 + p[3] = 0 + p[4] = 0 + p[5] = 0 + p[6] = 0 + step++ + return 7, nil + case 5: + n := copy(p, []byte{ + 1, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 0, 0, 0, 0, 0, 0, 0, 0, + }) + step++ + return n, nil + } + return 0, errors.New("timeout") + }, + } + req, err := reader.Read(r) + assert.NotNil(t, req) + assert.Nil(t, err) + assert.Equal(t, byte(1), req.Type) + assert.Equal(t, []byte{12, 34}, req.Data) + + req, err = reader.Read(r) + assert.NotNil(t, req) + assert.Nil(t, err) + assert.Equal(t, byte(22), req.Type) + assert.Equal(t, 0, len(req.Data)) + + req, err = reader.Read(r) + assert.NotNil(t, req) + assert.Nil(t, err) + assert.Equal(t, byte(1), req.Type) + assert.Equal(t, 0, len(req.Data)) + req, err = reader.Read(r) + assert.NotNil(t, req) + assert.Nil(t, err) + assert.Equal(t, byte(1), req.Type) + assert.Equal(t, 0, len(req.Data)) + req, err = reader.Read(r) + assert.NotNil(t, req) + assert.Nil(t, err) + assert.Equal(t, byte(1), req.Type) + assert.Equal(t, 0, len(req.Data)) +} diff --git a/coordinator/rpc.go b/coordinator/rpc.go index a4aa5eb..f2b4ce1 100644 --- a/coordinator/rpc.go +++ b/coordinator/rpc.go @@ -4,6 +4,9 @@ import ( "encoding/json" "errors" "fmt" + "regexp" + "time" + "github.com/angopher/chronus/coordinator/internal" "github.com/gogo/protobuf/proto" "github.com/influxdata/influxdb/models" @@ -11,8 +14,6 @@ import ( "github.com/influxdata/influxdb/query" "github.com/influxdata/influxdb/tsdb" "github.com/influxdata/influxql" - "regexp" - "time" ) //go:generate protoc --gogo_out=. internal/data.proto @@ -159,7 +160,9 @@ func (r *ExecuteStatementRequest) SetStatement(statement string) { func (r *ExecuteStatementRequest) Database() string { return r.pb.GetDatabase() } // SetDatabase sets the database name. -func (r *ExecuteStatementRequest) SetDatabase(database string) { r.pb.Database = proto.String(database) } +func (r *ExecuteStatementRequest) SetDatabase(database string) { + r.pb.Database = proto.String(database) +} // MarshalBinary encodes the object to a binary format. func (r *ExecuteStatementRequest) MarshalBinary() ([]byte, error) { @@ -685,9 +688,10 @@ func (r *CreateIteratorRequest) UnmarshalBinary(data []byte) error { // CreateIteratorResponse represents a response from remote iterator creation. type CreateIteratorResponse struct { - DataType influxql.DataType - SeriesN int - Err error + DataType influxql.DataType + SeriesN int + Err error + Termination []byte } // MarshalBinary encodes r to a binary format. diff --git a/coordinator/service.go b/coordinator/service.go index 3a97d16..56fe08b 100644 --- a/coordinator/service.go +++ b/coordinator/service.go @@ -1,66 +1,47 @@ package coordinator import ( - "context" "encoding" "encoding/binary" - "errors" "fmt" "io" "net" - "runtime/debug" "strings" "sync" - "sync/atomic" "time" + "github.com/angopher/chronus/coordinator/internal" + "github.com/angopher/chronus/coordinator/request" + errs "github.com/go-errors/errors" "github.com/influxdata/influxdb" "github.com/influxdata/influxdb/models" - "github.com/influxdata/influxdb/pkg/tracing" "github.com/influxdata/influxdb/query" - "github.com/influxdata/influxdb/tsdb" "github.com/influxdata/influxql" "go.uber.org/zap" "github.com/influxdata/influxdb/services/meta" ) +// Service and remote executor are designed for "forwarded" requests +// between shards/nodes. For better performance we separate the design +// into several phases: +// 1. Introduce connection pooling avoiding create/destroy connections frequently. +// 2. Introduce multiplexing to reduce the cost of connections further more. +// But we should also notice that more than one connection are needed to avoid flow control of TCP. +// Now we implement connection pooling first and keep an versioning api which enables +// data node declaring its running version. + // MaxMessageSize defines how large a message can be before we reject it const MaxMessageSize = 1024 * 1024 * 1024 // 1GB // MuxHeader is the header byte used in the TCP mux. const MuxHeader = 2 -// Statistics maintained by the cluster package -const ( - writeShardReq = "writeShardReq" - writeShardPointsReq = "writeShardPointsReq" - writeShardFail = "writeShardFail" - - createIteratorReq = "createIteratorReq" - createIteratorFail = "createIteratorFail" - - fieldDimensionsReq = "fieldDimensionsReq" - fieldDimensionsFail = "fieldDimensionsFail" - - tagKeysReq = "tagKeysReq" - tagKeysFail = "tagKeysFail" - - tagValuesReq = "tagValuesReq" - tagValuesFail = "tagValuesFail" - - measurementNamesReq = "measurementNamesReq" - measurementNamesFail = "measurementNamesFail" - - seriesCardinalityReq = "seriesCardinalityReq" - seriesCardinalityFail = "seriesCardinalityFail" - - iteratorCostReq = "iteratorCostReq" - iteratorCostFail = "iteratorCostFail" - - mapTypeReq = "mapTypeReq" - mapTypeFail = "mapTypeFail" -) +type ServerResponse interface { + SetCode(int) + SetMessage(string) + MarshalBinary() ([]byte, error) +} // Service processes data received over raw TCP connections. type Service struct { @@ -79,7 +60,7 @@ type Service struct { TaskManager *query.TaskManager Logger *zap.Logger - stats *InternalServiceStatistics + stats *internal.InternalServiceStatistics } // NewService returns a new instance of Service. @@ -87,7 +68,7 @@ func NewService(c Config) *Service { return &Service{ closing: make(chan struct{}), //Logger: log.New(os.Stderr, "[cluster] ", log.LstdFlags), - stats: &InternalServiceStatistics{}, + stats: &internal.InternalServiceStatistics{}, Logger: zap.NewNop(), } } @@ -96,7 +77,7 @@ func NewService(c Config) *Service { func (s *Service) Open() error { s.Logger.Info("Starting cluster service") - // Begin serving conections. + // Begin serving connections. s.wg.Add(1) go s.serve() @@ -108,72 +89,6 @@ func (s *Service) WithLogger(log *zap.Logger) { s.Logger = log.With(zap.String("service", "cluster")) } -type InternalServiceStatistics struct { - WriteShardReq int64 - WriteShardPointsReq int64 - WriteShardFail int64 - - CreateIteratorReq int64 - CreateIteratorFail int64 - - FieldDimensionsReq int64 - FieldDimensionsFail int64 - - TagKeysReq int64 - TagKeysFail int64 - - TagValuesReq int64 - TagValuesFail int64 - - MeasurementNamesReq int64 - MeasurementNamesFail int64 - - SeriesCardinalityReq int64 - SeriesCardinalityFail int64 - - IteratorCostReq int64 - IteratorCostFail int64 - - MapTypeReq int64 - MapTypeFail int64 -} - -func (w *Service) Statistics(tags map[string]string) []models.Statistic { - return []models.Statistic{{ - Name: "coordinator_service", - Tags: tags, - Values: map[string]interface{}{ - writeShardReq: atomic.LoadInt64(&w.stats.WriteShardReq), - writeShardPointsReq: atomic.LoadInt64(&w.stats.WriteShardPointsReq), - writeShardFail: atomic.LoadInt64(&w.stats.WriteShardFail), - - createIteratorReq: atomic.LoadInt64(&w.stats.CreateIteratorReq), - createIteratorFail: atomic.LoadInt64(&w.stats.CreateIteratorFail), - - fieldDimensionsReq: atomic.LoadInt64(&w.stats.FieldDimensionsReq), - fieldDimensionsFail: atomic.LoadInt64(&w.stats.FieldDimensionsFail), - - tagKeysReq: atomic.LoadInt64(&w.stats.TagKeysReq), - tagKeysFail: atomic.LoadInt64(&w.stats.TagKeysFail), - - tagValuesReq: atomic.LoadInt64(&w.stats.TagValuesReq), - tagValuesFail: atomic.LoadInt64(&w.stats.TagValuesFail), - - measurementNamesReq: atomic.LoadInt64(&w.stats.MeasurementNamesReq), - measurementNamesFail: atomic.LoadInt64(&w.stats.MeasurementNamesFail), - - seriesCardinalityReq: atomic.LoadInt64(&w.stats.SeriesCardinalityReq), - seriesCardinalityFail: atomic.LoadInt64(&w.stats.SeriesCardinalityFail), - - iteratorCostReq: atomic.LoadInt64(&w.stats.IteratorCostReq), - iteratorCostFail: atomic.LoadInt64(&w.stats.IteratorCostFail), - - mapTypeReq: atomic.LoadInt64(&w.stats.MapTypeReq), - mapTypeFail: atomic.LoadInt64(&w.stats.MapTypeFail), - }, - }} -} - // serve accepts connections from the listener and handles them. func (s *Service) serve() { defer s.wg.Done() @@ -203,8 +118,7 @@ func (s *Service) serve() { defer func() { s.wg.Done() if err := recover(); err != nil { - buf := debug.Stack() - s.Logger.Error("recover from panic", zap.String("stack", string(buf))) + s.Logger.Error("recover from panic", zap.String("stack", errs.Wrap(err, 2).ErrorStack())) } }() @@ -226,413 +140,147 @@ func (s *Service) Close() error { return nil } -// handleConn services an individual TCP connection. -func (s *Service) handleConn(conn net.Conn) { - // Ensure connection is closed when service is closed. - closing := make(chan struct{}) - defer close(closing) - go func() { - select { - case <-closing: - case <-s.closing: - } - conn.Close() - }() - - s.Logger.Info(fmt.Sprintf("accept remote connection from %+v", conn.RemoteAddr())) - defer func() { - s.Logger.Info(fmt.Sprintf("close remote connection from %+v", conn.RemoteAddr())) - }() - for { - // Read type-length-value. - typ, err := ReadType(conn) - if err != nil { - if strings.HasSuffix(err.Error(), "EOF") { - return - } - s.Logger.Error("unable to read type", zap.Error(err)) - return - } - - // Delegate message processing by type. - switch typ { - case writeShardRequestMessage: - buf, err := ReadLV(conn) - if err != nil { - s.Logger.Error("unable to read length-value", zap.Error(err)) - return - } - - atomic.AddInt64(&s.stats.WriteShardReq, 1) - err = s.processWriteShardRequest(buf) - if err != nil { - s.Logger.Error("process write shard error", zap.Error(err)) - } - s.writeShardResponse(conn, err) - case executeStatementRequestMessage: - buf, err := ReadLV(conn) - if err != nil { - s.Logger.Error("unable to read length-value", zap.Error(err)) - return - } - - err = s.processExecuteStatementRequest(buf) - if err != nil { - s.Logger.Error("process execute statement error", zap.Error(err)) - } - s.writeShardResponse(conn, err) - case createIteratorRequestMessage: - atomic.AddInt64(&s.stats.CreateIteratorReq, 1) - s.processCreateIteratorRequest(conn) - return - case fieldDimensionsRequestMessage: - atomic.AddInt64(&s.stats.FieldDimensionsReq, 1) - s.processFieldDimensionsRequest(conn) - return - case tagKeysRequestMessage: - atomic.AddInt64(&s.stats.TagKeysReq, 1) - s.processTagKeysRequest(conn) - case tagValuesRequestMessage: - atomic.AddInt64(&s.stats.TagValuesReq, 1) - s.processTagValuesRequest(conn) - return - case measurementNamesRequestMessage: - atomic.AddInt64(&s.stats.MeasurementNamesReq, 1) - s.processMeasurementNamesRequest(conn) - return - case seriesCardinalityRequestMessage: - atomic.AddInt64(&s.stats.SeriesCardinalityReq, 1) - s.processSeriesCardinalityRequest(conn) - return - case deleteSeriesRequestMessage: - s.processDeleteSeriesRequest(conn) - return - case deleteDatabaseRequestMessage: - s.processDeleteDatabaseRequest(conn) - return - case deleteMeasurementRequestMessage: - s.processDeleteMeasurementRequest(conn) - return - case iteratorCostRequestMessage: - atomic.AddInt64(&s.stats.IteratorCostReq, 1) - s.processIteratorCostRequest(conn) - return - case mapTypeRequestMessage: - atomic.AddInt64(&s.stats.MapTypeReq, 1) - s.processMapTypeRequest(conn) - return - case executeTaskManagerRequestMessage: - s.processTaskManagerRequest(conn) - return - default: - s.Logger.Error("cluster service message type not found", zap.Uint8("type", typ)) - } +func (s *Service) handleRequest(typ byte, data []byte) (respType byte, resp encoding.BinaryMarshaler, err error) { + // Delegate message processing by type. + switch typ { + case writeShardRequestMessage: + err = s.processWriteShardRequest(data) + respType = writeShardResponseMessage + resp = &WriteShardResponse{} + case executeStatementRequestMessage: + err = s.processExecuteStatementRequest(data) + respType = writeShardResponseMessage + resp = &WriteShardResponse{} + case fieldDimensionsRequestMessage: + respType = fieldDimensionsResponseMessage + resp, err = s.processFieldDimensionsRequest(data) + case tagKeysRequestMessage: + respType = tagKeysResponseMessage + resp, err = s.processTagKeysRequest(data) + case tagValuesRequestMessage: + respType = tagValuesResponseMessage + resp, err = s.processTagValuesRequest(data) + case measurementNamesRequestMessage: + respType = measurementNamesResponseMessage + resp, err = s.processMeasurementNamesRequest(data) + case seriesCardinalityRequestMessage: + respType = seriesCardinalityResponseMessage + resp, err = s.processSeriesCardinalityRequest(data) + case deleteSeriesRequestMessage: + respType = deleteSeriesResponseMessage + resp, err = s.processDeleteSeriesRequest(data) + case deleteDatabaseRequestMessage: + respType = deleteDatabaseResponseMessage + resp, err = s.processDeleteDatabaseRequest(data) + case deleteMeasurementRequestMessage: + respType = deleteMeasurementResponseMessage + resp, err = s.processDeleteMeasurementRequest(data) + case iteratorCostRequestMessage: + respType = iteratorCostResponseMessage + resp, err = s.processIteratorCostRequest(data) + case mapTypeRequestMessage: + respType = mapTypeResponseMessage + resp, err = s.processMapTypeRequest(data) + case executeTaskManagerRequestMessage: + respType = executeTaskManagerResponseMessage + resp, err = s.processTaskManagerRequest(data) + case testRequestMessage: + // do nothing + default: + s.Logger.Error("cluster service message type not found", zap.Uint8("type", typ)) } -} - -func (s *Service) processTaskManagerRequest(conn net.Conn) { - defer conn.Close() - - var resp TaskManagerStatementResponse - if err := func() error { - var req TaskManagerStatementRequest - if err := DecodeLV(conn, &req); err != nil { - return err - } - - stmt, err := influxql.ParseStatement(req.Statement()) - if err != nil { - return err - } - - recvCtx := &query.ExecutionContext{ - Context: context.Background(), - Results: make(chan *query.Result, 1), - } - err = s.TaskManager.ExecuteStatement(stmt, recvCtx) - if err != nil { - return err - } - resp.Result = *(<-recvCtx.Results) - return nil - }(); err != nil { - s.Logger.Error("s.processTaskManagerRequest fail", zap.Error(err)) - resp.Err = err.Error() + if err != nil { + s.Logger.Error("process request error", zap.Uint8("type", typ), zap.Error(err)) } - - if err := EncodeTLV(conn, executeTaskManagerResponseMessage, &resp); err != nil { - s.Logger.Error("s.processTaskManagerRequest EncodeTLV fail", zap.Error(err)) + if resp == nil { + return } -} - -func (s *Service) processMapTypeRequest(conn net.Conn) { - defer conn.Close() - var resp MapTypeResponse - if err := func() error { - var req MapTypeRequest - if err := DecodeLV(conn, &req); err != nil { - return err - } - - if len(req.Sources) == 0 { - return errors.New(fmt.Sprintf("bad request %+v: no sources", req)) - } - m := req.Sources[0].(*influxql.Measurement) - - sg := s.TSDBStore.ShardGroup(req.ShardIDs) - - var names []string - if m.Regex != nil && m.Name != "_series" && m.Name != "_fieldKeys" && m.Name != "_tagKeys" { - names = sg.MeasurementsByRegex(m.Regex.Val) + if serverResp, ok := resp.(ServerResponse); ok { + if err != nil { + serverResp.SetCode(1) + serverResp.SetMessage(err.Error()) } else { - names = []string{m.Name} + serverResp.SetCode(0) } - - typ := influxql.Unknown - for _, name := range names { - if m.SystemIterator != "" { - name = m.SystemIterator - } - t := sg.MapType(name, req.Field) - if typ.LessThan(t) { - typ = t - } - } - resp.DataType = typ - return nil - }(); err != nil { - atomic.AddInt64(&s.stats.MapTypeFail, 1) - s.Logger.Error("processMapTypeRequest fail", zap.Error(err)) - resp.Err = err.Error() - } - - if err := EncodeTLV(conn, mapTypeResponseMessage, &resp); err != nil { - atomic.AddInt64(&s.stats.MapTypeFail, 1) - s.Logger.Error("processMapTypeRequest EncodeTLV fail", zap.Error(err)) } + return } -func (s *Service) processIteratorCostRequest(conn net.Conn) { - defer conn.Close() - var resp IteratorCostResponse - if err := func() error { - var req IteratorCostRequest - if err := DecodeLV(conn, &req); err != nil { - return err - } - - if len(req.Sources) == 0 { - return errors.New(fmt.Sprintf("bad request %+v: no sources", req)) - } - m := req.Sources[0].(*influxql.Measurement) - opt := req.Opt - - var err error - sg := s.TSDBStore.ShardGroup(req.ShardIDs) - if m.Regex != nil { - resp.Cost, err = func() (query.IteratorCost, error) { - var costs query.IteratorCost - measurements := sg.MeasurementsByRegex(m.Regex.Val) - for _, measurement := range measurements { - c, err := sg.IteratorCost(measurement, opt) - if err != nil { - return c, err - } - costs = costs.Combine(c) - } - return costs, nil - }() - } else { - resp.Cost, err = sg.IteratorCost(m.Name, opt) - } - return err - }(); err != nil { - atomic.AddInt64(&s.stats.IteratorCostFail, 1) - s.Logger.Error("processIteratorCostRequest fail", zap.Error(err)) - resp.Err = err.Error() +func (s *Service) writeClusterResponse(conn io.Writer, respType byte, resp encoding.BinaryMarshaler) (err error) { + // Marshal response to binary. + buf, err := resp.MarshalBinary() + if err != nil { + s.Logger.Error("error marshalling response", zap.Uint8("respType", respType), zap.Error(err)) + return } - if err := EncodeTLV(conn, iteratorCostResponseMessage, &resp); err != nil { - atomic.AddInt64(&s.stats.IteratorCostFail, 1) - s.Logger.Error("processIteratorCostRequest EncodeTLV fail", zap.Error(err)) + // Write to connection. + if err = WriteTLV(conn, respType, buf); err != nil { + s.Logger.Error("WriteTLV fail", zap.Error(err)) } + return } -func (s *Service) processDeleteMeasurementRequest(conn net.Conn) { - defer conn.Close() - var resp DeleteMeasurementResponse - if err := func() error { - var req DeleteMeasurementRequest - if err := DecodeLV(conn, &req); err != nil { - return err - } - return s.TSDBStore.DeleteMeasurement(req.Database, req.Name) - }(); err != nil { - s.Logger.Error("processDeleteMeasurementRequest fail", zap.Error(err)) - resp.Err = err.Error() +func canContinue(err error) bool { + if err == io.EOF || err.Error() == "use of closed network connection" { + return false } - - if err := EncodeTLV(conn, deleteMeasurementResponseMessage, &resp); err != nil { - s.Logger.Error("processDeleteMeasurementRequest EncodeTLV fail", zap.Error(err)) + if err, ok := err.(net.Error); ok && err.Timeout() { + return true } + return false } -func (s *Service) processDeleteDatabaseRequest(conn net.Conn) { - defer conn.Close() - var resp DeleteDatabaseResponse - if err := func() error { - var req DeleteDatabaseRequest - if err := DecodeLV(conn, &req); err != nil { - return err +// handleConn services an individual TCP connection. +func (s *Service) handleConn(conn net.Conn) { + // Ensure connection is closed when service is closed. + defer func() { + defer conn.Close() + s.Logger.Info(fmt.Sprintf("close remote connection from %+v", conn.RemoteAddr())) + }() + s.Logger.Info(fmt.Sprintf("accept remote connection from %+v", conn.RemoteAddr())) + requestReader := &request.ClusterMessageReader{} + for { + select { + case <-s.closing: + return + default: } - return s.TSDBStore.DeleteDatabase(req.Database) - }(); err != nil { - s.Logger.Error("processDeleteDatabaseRequest fail", zap.Error(err)) - resp.Err = err.Error() - } - - if err := EncodeTLV(conn, deleteDatabaseResponseMessage, &resp); err != nil { - s.Logger.Error("processDeleteDatabaseRequest EncodeTLV", zap.Error(err)) - } -} - -func (s *Service) processDeleteSeriesRequest(conn net.Conn) { - defer conn.Close() - var resp DeleteSeriesResponse - if err := func() error { - var req DeleteSeriesRequest - if err := DecodeLV(conn, &req); err != nil { - return err + conn.SetReadDeadline(time.Now().Add(500 * time.Millisecond)) + req, err := requestReader.Read(conn) + conn.SetReadDeadline(time.Time{}) + if err != nil { + if canContinue(err) { + continue + } + if err != io.EOF { + s.Logger.Error("read error", zap.Error(err)) + } + break } - - cond := influxql.Reduce(req.Cond, &influxql.NowValuer{Now: time.Now().UTC()}) - err := s.TSDBStore.DeleteSeries(req.Database, req.Sources, cond) - return err - }(); err != nil { - s.Logger.Error("processDeleteSeriesRequest fail", zap.Error(err)) - resp.Err = err.Error() - } - - if err := EncodeTLV(conn, deleteSeriesResponseMessage, &resp); err != nil { - s.Logger.Error("processDeleteSeriesRequest EncodeTLV fail", zap.Error(err)) - } -} - -func (s *Service) processSeriesCardinalityRequest(conn net.Conn) { - defer conn.Close() - var resp SeriesCardinalityResponse - if err := func() error { - var req SeriesCardinalityRequest - if err := DecodeLV(conn, &req); err != nil { - return err + if req == nil { + continue } - - n, err := s.TSDBStore.SeriesCardinality(req.Database) - resp.N = n - return err - }(); err != nil { - atomic.AddInt64(&s.stats.SeriesCardinalityFail, 1) - s.Logger.Error("processSeriesCardinalityRequest fail", zap.Error(err)) - resp.Err = err.Error() - } - - if err := EncodeTLV(conn, seriesCardinalityResponseMessage, &resp); err != nil { - atomic.AddInt64(&s.stats.SeriesCardinalityFail, 1) - s.Logger.Error("processSeriesCardinalityRequest EncodeTLV", zap.Error(err)) - } -} - -func (s *Service) processMeasurementNamesRequest(conn net.Conn) { - defer conn.Close() - - var resp MeasurementNamesResponse - if err := func() error { - var req MeasurementNamesRequest - if err := DecodeLV(conn, &req); err != nil { - return err + if req.Type == createIteratorRequestMessage { + // iterator is different from other requests + ioError := s.processCreateIteratorRequest(conn, req.Data) + if ioError { + break + } + continue } - - var err error - resp.Names, err = s.TSDBStore.MeasurementNames(nil, req.Database, req.Cond) - return err - }(); err != nil { - s.Logger.Error("processMeasurementNamesRequest fail", zap.Error(err)) - atomic.AddInt64(&s.stats.MeasurementNamesFail, 1) - resp.Err = err.Error() - } - - if err := EncodeTLV(conn, measurementNamesResponseMessage, &resp); err != nil { - atomic.AddInt64(&s.stats.MeasurementNamesFail, 1) - s.Logger.Error("processMeasurementNamesRequest EncodeTLV fail", zap.Error(err)) - } -} - -func (s *Service) processTagKeysRequest(conn net.Conn) { - defer conn.Close() - - var resp TagKeysResponse - if err := func() error { - var req TagKeysRequest - if err := DecodeLV(conn, &req); err != nil { - return err + respType, resp, err := s.handleRequest(req.Type, req.Data) + if resp == nil { + continue } - - var err error - resp.TagKeys, err = s.TSDBStore.TagKeys(nil, req.ShardIDs, req.Cond) - return err - }(); err != nil { - atomic.AddInt64(&s.stats.TagKeysFail, 1) - s.Logger.Error("processTagKeysRequest fail", zap.Error(err)) - resp.Err = err.Error() - } - - if err := EncodeTLV(conn, tagKeysResponseMessage, &resp); err != nil { - atomic.AddInt64(&s.stats.TagKeysFail, 1) - s.Logger.Error("processTagKeysRequest EncodeTLV", zap.Error(err)) - } -} - -func (s *Service) processTagValuesRequest(conn net.Conn) { - defer conn.Close() - - var resp TagValuesResponse - if err := func() error { - var req TagValuesRequest - if err := DecodeLV(conn, &req); err != nil { - return err + err = s.writeClusterResponse(conn, respType, resp) + if err != nil { + // close conn due to error in writing + break } - - var err error - resp.TagValues, err = s.TSDBStore.TagValues(nil, req.ShardIDs, req.Cond) - return err - }(); err != nil { - atomic.AddInt64(&s.stats.TagValuesFail, 1) - s.Logger.Error("processTagValuesRequest fail", zap.Error(err)) - resp.Err = err.Error() - } - - if err := EncodeTLV(conn, tagValuesResponseMessage, &resp); err != nil { - atomic.AddInt64(&s.stats.TagValuesFail, 1) - s.Logger.Error("processTagValuesRequest EncodeTLV fail", zap.Error(err)) } } -func (s *Service) processExecuteStatementRequest(buf []byte) error { - // Unmarshal the request. - var req ExecuteStatementRequest - if err := req.UnmarshalBinary(buf); err != nil { - return err - } - - // Parse the InfluxQL statement. - stmt, err := influxql.ParseStatement(req.Statement()) - if err != nil { - return err - } - - return s.executeStatement(stmt, req.Database()) -} - func (s *Service) executeStatement(stmt influxql.Statement, database string) error { switch t := stmt.(type) { case *influxql.DropDatabaseStatement: @@ -648,278 +296,43 @@ func (s *Service) executeStatement(stmt influxql.Statement, database string) err } } -func (s *Service) processWriteShardRequest(buf []byte) error { - // Build request - var req WriteShardRequest - if err := req.UnmarshalBinary(buf); err != nil { - return err - } - - points := req.Points() - atomic.AddInt64(&s.stats.WriteShardPointsReq, int64(len(points))) - err := s.TSDBStore.WriteToShard(req.ShardID(), points) - - // We may have received a write for a shard that we don't have locally because the - // sending node may have just created the shard (via the metastore) and the write - // arrived before the local store could create the shard. In this case, we need - // to check the metastore to determine what database and retention policy this - // shard should reside within. - if err == tsdb.ErrShardNotFound { - db, rp := req.Database(), req.RetentionPolicy() - if db == "" || rp == "" { - s.Logger.Error("drop write request: no database or rentention policy received\n", - zap.Uint64("shard", req.ShardID())) - return nil - } - - err = s.TSDBStore.CreateShard(req.Database(), req.RetentionPolicy(), req.ShardID(), true) //enable what mean? - if err != nil { - atomic.AddInt64(&s.stats.WriteShardFail, 1) - return fmt.Errorf("create shard %d: %s", req.ShardID(), err) - } - - err = s.TSDBStore.WriteToShard(req.ShardID(), points) - if err != nil { - atomic.AddInt64(&s.stats.WriteShardFail, 1) - return fmt.Errorf("write shard %d: %s", req.ShardID(), err) - } - } - - if err != nil { - atomic.AddInt64(&s.stats.WriteShardFail, 1) - return fmt.Errorf("write shard %d: %s", req.ShardID(), err) - } - - return nil -} - -func (s *Service) writeShardResponse(w io.Writer, e error) { - // Build response. - var resp WriteShardResponse - if e != nil { - resp.SetCode(1) - resp.SetMessage(e.Error()) - } else { - resp.SetCode(0) - } - - // Marshal response to binary. - buf, err := resp.MarshalBinary() - if err != nil { - s.Logger.Error("error marshalling shard response", zap.Error(err)) - return - } - - // Write to connection. - if err := WriteTLV(w, writeShardResponseMessage, buf); err != nil { - s.Logger.Error("write shard WriteTLV fail", zap.Error(err)) - } -} - -func (s *Service) processCreateIteratorRequest(conn net.Conn) { - defer conn.Close() - - var itr query.Iterator - var trace *tracing.Trace - var span *tracing.Span - if err := func() error { - // Parse request. - var req CreateIteratorRequest - if err := DecodeLV(conn, &req); err != nil { - return err - } - - ctx := context.Background() - if req.SpanContex != nil { - trace, span = tracing.NewTraceFromSpan(fmt.Sprintf("remote_node_id: %d", s.Node.ID), *req.SpanContex) - ctx = tracing.NewContextWithTrace(ctx, trace) - ctx = tracing.NewContextWithSpan(ctx, span) - //var aux query.Iterators - //ctx = query.NewContextWithIterators(ctx, &aux) - } - - var err error - - //TODO:求证是否直接使用&req.Measurement会不会导致内存错误 - m := new(influxql.Measurement) - *m = req.Measurement - - sg := s.TSDBStore.ShardGroup(req.ShardIDs) - if m.Regex != nil { - measurements := sg.MeasurementsByRegex(m.Regex.Val) - inputs := make([]query.Iterator, 0, len(measurements)) - if err := func() error { - for _, measurement := range measurements { - mm := m.Clone() - mm.Name = measurement - input, err := sg.CreateIterator(ctx, mm, req.Opt) - if err != nil { - return err - } - inputs = append(inputs, input) - } - return nil - }(); err != nil { - query.Iterators(inputs).Close() - return err - } - - itr, err = query.Iterators(inputs).Merge(req.Opt) - } else { - itr, err = sg.CreateIterator(ctx, m, req.Opt) - } - - if err != nil { - return err - } - // Generate a single iterator from all shards. - //i, err := influxql.IteratorCreators(ics).CreateIterator(req.Opt) - - return nil - }(); err != nil { - atomic.AddInt64(&s.stats.CreateIteratorFail, 1) - if itr != nil { - itr.Close() - } - s.Logger.Error("error reading CreateIterator request fail", zap.Error(err)) - if err = EncodeTLV(conn, createIteratorResponseMessage, &CreateIteratorResponse{Err: err}); err != nil { - s.Logger.Error("CreateIteratorRequest EncodeTLV fail", zap.Error(err)) - } - return - } - - dataType := influxql.Unknown - switch itr.(type) { - case query.FloatIterator: - dataType = influxql.Float - case query.IntegerIterator: - dataType = influxql.Integer - case query.StringIterator: - dataType = influxql.String - case query.BooleanIterator: - dataType = influxql.Boolean - } - - seriesN := 0 - if itr != nil { - seriesN = itr.Stats().SeriesN - } - // Encode success response. - if err := EncodeTLV(conn, createIteratorResponseMessage, &CreateIteratorResponse{DataType: dataType, SeriesN: seriesN}); err != nil { - s.Logger.Error("error writing CreateIterator response, EncodeTLV fail", zap.Error(err)) - atomic.AddInt64(&s.stats.CreateIteratorFail, 1) - return - } - - // Exit if no iterator was produced. - if itr == nil { - return - } - - // Stream iterator to connection. - if err := query.NewIteratorEncoder(conn).EncodeIterator(itr); err != nil { - s.Logger.Error("encoding CreateIterator iterator fail", zap.Error(err)) - atomic.AddInt64(&s.stats.CreateIteratorFail, 1) - return - } - - itr.Close() - - if trace != nil { - span.Finish() - if err := query.NewIteratorEncoder(conn).EncodeTrace(trace); err != nil { - s.Logger.Error("EncodeTrace fail", zap.Error(err)) - atomic.AddInt64(&s.stats.CreateIteratorFail, 1) - return - } - } -} - -func (s *Service) processFieldDimensionsRequest(conn net.Conn) { - var fields map[string]influxql.DataType - var dimensions map[string]struct{} - if err := func() error { - // Parse request. - var req FieldDimensionsRequest - if err := DecodeLV(conn, &req); err != nil { - return err - } - - // Generate a single iterator from all shards. - measurements := make([]string, 0) - ms := req.Sources.Measurements() - for _, m := range ms { - if m.Regex != nil { - measurements = s.TSDBStore.ShardGroup(req.ShardIDs).MeasurementsByRegex(m.Regex.Val) - } else { - measurements = append(measurements, m.Name) - } - } - - f, d, err := s.TSDBStore.ShardGroup(req.ShardIDs).FieldDimensions(measurements) - if err != nil { - return err - } - fields, dimensions = f, d - - return nil - }(); err != nil { - atomic.AddInt64(&s.stats.FieldDimensionsFail, 1) - s.Logger.Error("error reading FieldDimensions request", zap.Error(err)) - EncodeTLV(conn, fieldDimensionsResponseMessage, &FieldDimensionsResponse{Err: err}) - return - } - - // Encode success response. - if err := EncodeTLV(conn, fieldDimensionsResponseMessage, &FieldDimensionsResponse{ - Fields: fields, - Dimensions: dimensions, - }); err != nil { - atomic.AddInt64(&s.stats.FieldDimensionsFail, 1) - s.Logger.Error("error writing FieldDimensions response", zap.Error(err)) - return - } -} - -// ReadTLV reads a type-length-value record from r. -func ReadTLV(r io.Reader) (byte, []byte, error) { - typ, err := ReadType(r) - if err != nil { - return 0, nil, err - } - - buf, err := ReadLV(r) - if err != nil { - return 0, nil, err - } - return typ, buf, err +func (s *Service) Statistics(tags map[string]string) []models.Statistic { + return internal.Statistics(s.stats, tags) } // ReadType reads the type from a TLV record. func ReadType(r io.Reader) (byte, error) { var typ [1]byte if _, err := io.ReadFull(r, typ[:]); err != nil { - return 0, fmt.Errorf("read message type: %s", err) + return 0, err } return typ[0], nil } // ReadLV reads the length-value from a TLV record. -func ReadLV(r io.Reader) ([]byte, error) { +func ReadLV(conn net.Conn, timeout time.Duration) ([]byte, error) { // Read the size of the message. var sz int64 - if err := binary.Read(r, binary.BigEndian, &sz); err != nil { - return nil, fmt.Errorf("read message size: %s", err) + conn.SetReadDeadline(time.Now().Add(timeout)) + defer conn.SetReadDeadline(time.Time{}) + if err := binary.Read(conn, binary.BigEndian, &sz); err != nil { + return nil, err } if sz >= MaxMessageSize { return nil, fmt.Errorf("max message size of %d exceeded: %d", MaxMessageSize, sz) } + if sz == 0 { + // empty msg + return []byte{}, nil + } + // Read the value. buf := make([]byte, sz) - if _, err := io.ReadFull(r, buf); err != nil { - return nil, fmt.Errorf("read message value: %s", err) + conn.SetReadDeadline(time.Now().Add(timeout)) + if _, err := io.ReadFull(conn, buf); err != nil { + return nil, err } return buf, nil @@ -939,7 +352,7 @@ func WriteTLV(w io.Writer, typ byte, buf []byte) error { // WriteType writes the type in a TLV record to w. func WriteType(w io.Writer, typ byte) error { if _, err := w.Write([]byte{typ}); err != nil { - return fmt.Errorf("write message type: %s", err) + return err } return nil } @@ -948,12 +361,12 @@ func WriteType(w io.Writer, typ byte) error { func WriteLV(w io.Writer, buf []byte) error { // Write the size of the message. if err := binary.Write(w, binary.BigEndian, int64(len(buf))); err != nil { - return fmt.Errorf("write message size: %s", err) + return err } // Write the value. if _, err := w.Write(buf); err != nil { - return fmt.Errorf("write message value: %s", err) + return err } return nil } @@ -981,28 +394,3 @@ func EncodeLV(w io.Writer, v encoding.BinaryMarshaler) error { } return nil } - -// DecodeTLV reads the type-length-value record from r and unmarshals it into v. -func DecodeTLV(r io.Reader, v encoding.BinaryUnmarshaler) (typ byte, err error) { - typ, err = ReadType(r) - if err != nil { - return 0, err - } - if err := DecodeLV(r, v); err != nil { - return 0, err - } - return typ, nil -} - -// DecodeLV reads the length-value record from r and unmarshals it into v. -func DecodeLV(r io.Reader, v encoding.BinaryUnmarshaler) error { - buf, err := ReadLV(r) - if err != nil { - return err - } - - if err := v.UnmarshalBinary(buf); err != nil { - return err - } - return nil -} diff --git a/coordinator/service_handlers.go b/coordinator/service_handlers.go new file mode 100644 index 0000000..3175240 --- /dev/null +++ b/coordinator/service_handlers.go @@ -0,0 +1,513 @@ +package coordinator + +import ( + "context" + "errors" + "fmt" + "io" + "sync/atomic" + "time" + + "github.com/angopher/chronus/x" + "github.com/influxdata/influxdb/pkg/tracing" + "github.com/influxdata/influxdb/query" + "github.com/influxdata/influxdb/tsdb" + "github.com/influxdata/influxql" + "go.uber.org/zap" +) + +func (s *Service) processTaskManagerRequest(buf []byte) (*TaskManagerStatementResponse, error) { + var ( + resp TaskManagerStatementResponse + err error + ) + if err = func() error { + var req TaskManagerStatementRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + stmt, err := influxql.ParseStatement(req.Statement()) + if err != nil { + return err + } + + recvCtx := &query.ExecutionContext{ + Context: context.Background(), + Results: make(chan *query.Result, 1), + } + err = s.TaskManager.ExecuteStatement(stmt, recvCtx) + if err != nil { + return err + } + resp.Result = *(<-recvCtx.Results) + return nil + }(); err != nil { + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processMapTypeRequest(buf []byte) (*MapTypeResponse, error) { + var ( + resp MapTypeResponse + err error + ) + atomic.AddInt64(&s.stats.MapTypeReq, 1) + if err = func() error { + var req MapTypeRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + if len(req.Sources) == 0 { + return errors.New(fmt.Sprintf("bad request %+v: no sources", req)) + } + m := req.Sources[0].(*influxql.Measurement) + + sg := s.TSDBStore.ShardGroup(req.ShardIDs) + + var names []string + if m.Regex != nil && m.Name != "_series" && m.Name != "_fieldKeys" && m.Name != "_tagKeys" { + names = sg.MeasurementsByRegex(m.Regex.Val) + } else { + names = []string{m.Name} + } + + typ := influxql.Unknown + for _, name := range names { + if m.SystemIterator != "" { + name = m.SystemIterator + } + t := sg.MapType(name, req.Field) + if typ.LessThan(t) { + typ = t + } + } + resp.DataType = typ + return nil + }(); err != nil { + atomic.AddInt64(&s.stats.MapTypeFail, 1) + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processIteratorCostRequest(buf []byte) (*IteratorCostResponse, error) { + var ( + resp IteratorCostResponse + err error + ) + atomic.AddInt64(&s.stats.IteratorCostReq, 1) + if err = func() error { + var req IteratorCostRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + if len(req.Sources) == 0 { + return errors.New(fmt.Sprintf("bad request %+v: no sources", req)) + } + m := req.Sources[0].(*influxql.Measurement) + opt := req.Opt + + sg := s.TSDBStore.ShardGroup(req.ShardIDs) + if m.Regex != nil { + resp.Cost, err = func() (query.IteratorCost, error) { + var costs query.IteratorCost + measurements := sg.MeasurementsByRegex(m.Regex.Val) + for _, measurement := range measurements { + c, err := sg.IteratorCost(measurement, opt) + if err != nil { + return c, err + } + costs = costs.Combine(c) + } + return costs, nil + }() + } else { + resp.Cost, err = sg.IteratorCost(m.Name, opt) + } + return err + }(); err != nil { + atomic.AddInt64(&s.stats.IteratorCostFail, 1) + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processDeleteMeasurementRequest(buf []byte) (*DeleteMeasurementResponse, error) { + var ( + resp DeleteMeasurementResponse + err error + ) + if err = func() error { + var req DeleteMeasurementRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + return s.TSDBStore.DeleteMeasurement(req.Database, req.Name) + }(); err != nil { + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processDeleteDatabaseRequest(buf []byte) (*DeleteDatabaseResponse, error) { + var ( + resp DeleteDatabaseResponse + err error + ) + if err = func() error { + var req DeleteDatabaseRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + return s.TSDBStore.DeleteDatabase(req.Database) + }(); err != nil { + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processDeleteSeriesRequest(buf []byte) (*DeleteSeriesResponse, error) { + var ( + resp DeleteSeriesResponse + err error + ) + if err = func() error { + var req DeleteSeriesRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + cond := influxql.Reduce(req.Cond, &influxql.NowValuer{Now: time.Now().UTC()}) + err := s.TSDBStore.DeleteSeries(req.Database, req.Sources, cond) + return err + }(); err != nil { + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processSeriesCardinalityRequest(buf []byte) (*SeriesCardinalityResponse, error) { + var ( + resp SeriesCardinalityResponse + err error + ) + atomic.AddInt64(&s.stats.SeriesCardinalityReq, 1) + if err = func() error { + var req SeriesCardinalityRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + n, err := s.TSDBStore.SeriesCardinality(req.Database) + resp.N = n + return err + }(); err != nil { + atomic.AddInt64(&s.stats.SeriesCardinalityFail, 1) + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processMeasurementNamesRequest(buf []byte) (*MeasurementNamesResponse, error) { + var ( + resp MeasurementNamesResponse + err error + ) + if err = func() error { + var req MeasurementNamesRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + var err error + resp.Names, err = s.TSDBStore.MeasurementNames(nil, req.Database, req.Cond) + return err + }(); err != nil { + atomic.AddInt64(&s.stats.MeasurementNamesFail, 1) + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processTagKeysRequest(buf []byte) (*TagKeysResponse, error) { + var ( + resp TagKeysResponse + err error + ) + atomic.AddInt64(&s.stats.TagKeysReq, 1) + if err = func() error { + var req TagKeysRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + var err error + resp.TagKeys, err = s.TSDBStore.TagKeys(nil, req.ShardIDs, req.Cond) + return err + }(); err != nil { + atomic.AddInt64(&s.stats.TagKeysFail, 1) + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processTagValuesRequest(buf []byte) (*TagValuesResponse, error) { + var ( + resp TagValuesResponse + err error + ) + atomic.AddInt64(&s.stats.TagValuesReq, 1) + if err = func() error { + var req TagValuesRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + var err error + resp.TagValues, err = s.TSDBStore.TagValues(nil, req.ShardIDs, req.Cond) + return err + }(); err != nil { + atomic.AddInt64(&s.stats.TagValuesFail, 1) + resp.Err = err.Error() + } + return &resp, err +} + +func (s *Service) processExecuteStatementRequest(buf []byte) error { + // Unmarshal the request. + var req ExecuteStatementRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + // Parse the InfluxQL statement. + stmt, err := influxql.ParseStatement(req.Statement()) + if err != nil { + return err + } + + return s.executeStatement(stmt, req.Database()) +} + +func (s *Service) processWriteShardRequest(buf []byte) error { + // Build request + var req WriteShardRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + points := req.Points() + // stats + atomic.AddInt64(&s.stats.WriteShardReq, 1) + atomic.AddInt64(&s.stats.WriteShardPointsReq, int64(len(points))) + err := s.TSDBStore.WriteToShard(req.ShardID(), points) + + // We may have received a write for a shard that we don't have locally because the + // sending node may have just created the shard (via the metastore) and the write + // arrived before the local store could create the shard. In this case, we need + // to check the metastore to determine what database and retention policy this + // shard should reside within. + if err == tsdb.ErrShardNotFound { + db, rp := req.Database(), req.RetentionPolicy() + if db == "" || rp == "" { + s.Logger.Error("drop write request: no database or retention policy received\n", + zap.Uint64("shard", req.ShardID())) + return nil + } + + err = s.TSDBStore.CreateShard(req.Database(), req.RetentionPolicy(), req.ShardID(), true) //enable what mean? + if err != nil { + atomic.AddInt64(&s.stats.WriteShardFail, 1) + return fmt.Errorf("create shard %d: %s", req.ShardID(), err) + } + + err = s.TSDBStore.WriteToShard(req.ShardID(), points) + if err != nil { + atomic.AddInt64(&s.stats.WriteShardFail, 1) + return fmt.Errorf("write shard %d: %s", req.ShardID(), err) + } + } + + if err != nil { + atomic.AddInt64(&s.stats.WriteShardFail, 1) + return fmt.Errorf("write shard %d: %s", req.ShardID(), err) + } + + return nil +} + +func (s *Service) processCreateIteratorRequest(conn io.ReadWriter, buf []byte) (ioError bool) { + var itr query.Iterator + var trace *tracing.Trace + var span *tracing.Span + respType := createIteratorResponseMessage + if err := func() error { + // Parse request. + var req CreateIteratorRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + ctx := context.Background() + if req.SpanContex != nil { + trace, span = tracing.NewTraceFromSpan(fmt.Sprintf("remote_node_id: %d", s.Node.ID), *req.SpanContex) + ctx = tracing.NewContextWithTrace(ctx, trace) + ctx = tracing.NewContextWithSpan(ctx, span) + //var aux query.Iterators + //ctx = query.NewContextWithIterators(ctx, &aux) + } + + var err error + + //TODO:求证是否直接使用&req.Measurement会不会导致内存错误 + m := new(influxql.Measurement) + *m = req.Measurement + + sg := s.TSDBStore.ShardGroup(req.ShardIDs) + if m.Regex != nil { + measurements := sg.MeasurementsByRegex(m.Regex.Val) + inputs := make([]query.Iterator, 0, len(measurements)) + if err := func() error { + for _, measurement := range measurements { + mm := m.Clone() + mm.Name = measurement + input, err := sg.CreateIterator(ctx, mm, req.Opt) + if err != nil { + return err + } + inputs = append(inputs, input) + } + return nil + }(); err != nil { + query.Iterators(inputs).Close() + return err + } + + itr, err = query.Iterators(inputs).Merge(req.Opt) + } else { + itr, err = sg.CreateIterator(ctx, m, req.Opt) + } + + if err != nil { + return err + } + // Generate a single iterator from all shards. + //i, err := influxql.IteratorCreators(ics).CreateIterator(req.Opt) + + return nil + }(); err != nil { + atomic.AddInt64(&s.stats.CreateIteratorFail, 1) + if itr != nil { + itr.Close() + } + s.Logger.Error("error reading CreateIterator request fail", zap.Error(err)) + if err = EncodeTLV(conn, respType, &CreateIteratorResponse{Err: err}); err != nil { + s.Logger.Error("CreateIteratorRequest EncodeTLV fail", zap.Error(err)) + ioError = true + } + return + } + + dataType := influxql.Unknown + switch itr.(type) { + case query.FloatIterator: + dataType = influxql.Float + case query.IntegerIterator: + dataType = influxql.Integer + case query.StringIterator: + dataType = influxql.String + case query.BooleanIterator: + dataType = influxql.Boolean + } + + seriesN := 0 + if itr != nil { + defer itr.Close() + seriesN = itr.Stats().SeriesN + } + // Encode success response. + itrTerminator := x.RandBytes(8) + if err := EncodeTLV(conn, respType, &CreateIteratorResponse{DataType: dataType, SeriesN: seriesN, Termination: itrTerminator}); err != nil { + s.Logger.Error("error writing CreateIterator response, EncodeTLV fail", zap.Error(err)) + atomic.AddInt64(&s.stats.CreateIteratorFail, 1) + ioError = true + return + } + defer func() { + // Write termination of iterator + if _, err := conn.Write(itrTerminator); err != nil { + ioError = true + } + }() + + // Exit if no iterator was produced. + if itr == nil { + return + } + + // Stream iterator to connection. + encoder := query.NewIteratorEncoder(conn) + if err := encoder.EncodeIterator(itr); err != nil { + s.Logger.Error("encoding CreateIterator iterator fail", zap.Error(err)) + atomic.AddInt64(&s.stats.CreateIteratorFail, 1) + ioError = true + return + } + + if trace != nil { + span.Finish() + if err := encoder.EncodeTrace(trace); err != nil { + s.Logger.Error("EncodeTrace fail", zap.Error(err)) + atomic.AddInt64(&s.stats.CreateIteratorFail, 1) + ioError = true + return + } + } + return +} + +func (s *Service) processFieldDimensionsRequest(buf []byte) (*FieldDimensionsResponse, error) { + var ( + err error + ) + var fields map[string]influxql.DataType + var dimensions map[string]struct{} + atomic.AddInt64(&s.stats.FieldDimensionsReq, 1) + if err = func() error { + // Parse request. + var req FieldDimensionsRequest + if err := req.UnmarshalBinary(buf); err != nil { + return err + } + + // Generate a single iterator from all shards. + measurements := make([]string, 0) + ms := req.Sources.Measurements() + for _, m := range ms { + if m.Regex != nil { + measurements = s.TSDBStore.ShardGroup(req.ShardIDs).MeasurementsByRegex(m.Regex.Val) + } else { + measurements = append(measurements, m.Name) + } + } + + f, d, err := s.TSDBStore.ShardGroup(req.ShardIDs).FieldDimensions(measurements) + if err != nil { + return err + } + fields, dimensions = f, d + + return nil + }(); err != nil { + atomic.AddInt64(&s.stats.FieldDimensionsFail, 1) + return &FieldDimensionsResponse{Err: err}, err + } + return &FieldDimensionsResponse{ + Fields: fields, + Dimensions: dimensions, + }, err +} diff --git a/coordinator/shard_writer.go b/coordinator/shard_writer.go index 8ca2bad..636948b 100644 --- a/coordinator/shard_writer.go +++ b/coordinator/shard_writer.go @@ -2,11 +2,12 @@ package coordinator import ( "fmt" - "net" "time" + "github.com/angopher/chronus/coordinator/request" "github.com/influxdata/influxdb/models" "github.com/influxdata/influxdb/services/meta" + "go.uber.org/zap" ) const ( @@ -54,13 +55,15 @@ const ( executeTaskManagerRequestMessage executeTaskManagerResponseMessage + + testRequestMessage // one way message ) // ShardWriter writes a set of points to a shard. type ShardWriter struct { - pool *clientPool - timeout time.Duration - maxConnections int + pool *ClientPool + timeout time.Duration + logger *zap.Logger MetaClient interface { DataNode(id uint64) (ni *meta.NodeInfo, err error) @@ -69,28 +72,25 @@ type ShardWriter struct { } // NewShardWriter returns a new instance of ShardWriter. -func NewShardWriter(timeout time.Duration, maxConnections int) *ShardWriter { +func NewShardWriter(timeout time.Duration, pool *ClientPool) *ShardWriter { return &ShardWriter{ - pool: newClientPool(), - timeout: timeout, - maxConnections: maxConnections, + pool: pool, + timeout: timeout, + logger: zap.NewNop(), } } +func (w *ShardWriter) WithLogger(logger *zap.Logger) { + w.logger = logger.With(zap.String("service", "ShardWriter")) +} + // WriteShard writes time series points to a shard func (w *ShardWriter) WriteShard(shardID, ownerID uint64, points []models.Point) error { - c, err := w.dial(ownerID) + conn, err := getConnWithRetry(w.pool, ownerID, w.logger) if err != nil { return err } - - conn, ok := c.(*pooledConn) - if !ok { - panic("wrong connection type") - } - defer func(conn net.Conn) { - conn.Close() // return to pool - }(conn) + defer conn.Close() // Determine the location of this shard and whether it still exists db, rp, sgi := w.MetaClient.ShardOwner(shardID) @@ -101,15 +101,15 @@ func (w *ShardWriter) WriteShard(shardID, ownerID uint64, points []models.Point) return nil } - // Build write request. - var request WriteShardRequest - request.SetShardID(shardID) - request.SetDatabase(db) - request.SetRetentionPolicy(rp) - request.AddPoints(points) + // Build write writeReq. + var writeReq WriteShardRequest + writeReq.SetShardID(shardID) + writeReq.SetDatabase(db) + writeReq.SetRetentionPolicy(rp) + writeReq.AddPoints(points) // Marshal into protocol buffers. - buf, err := request.MarshalBinary() + buf, err := writeReq.MarshalBinary() if err != nil { return err } @@ -122,16 +122,18 @@ func (w *ShardWriter) WriteShard(shardID, ownerID uint64, points []models.Point) } // Read the response. + requestReader := &request.ClusterMessageReader{} conn.SetReadDeadline(time.Now().Add(w.timeout)) - _, buf, err = ReadTLV(conn) + resp, err := requestReader.Read(conn) if err != nil { conn.MarkUnusable() return err } + conn.SetDeadline(time.Time{}) // Unmarshal response. var response WriteShardResponse - if err := response.UnmarshalBinary(buf); err != nil { + if err := response.UnmarshalBinary(resp.Data); err != nil { return err } @@ -142,22 +144,6 @@ func (w *ShardWriter) WriteShard(shardID, ownerID uint64, points []models.Point) return nil } -func (w *ShardWriter) dial(nodeID uint64) (net.Conn, error) { - // If we don't have a connection pool for that addr yet, create one - _, ok := w.pool.getPool(nodeID) - if !ok { - factory := &connFactory{nodeID: nodeID, clientPool: w.pool, timeout: w.timeout} - factory.metaClient = w.MetaClient - - p, err := NewBoundedPool(1, w.maxConnections, w.timeout, factory.dial) - if err != nil { - return nil, err - } - w.pool.setPool(nodeID, p) - } - return w.pool.conn(nodeID) -} - // Close closes ShardWriter's pool func (w *ShardWriter) Close() error { if w.pool == nil { @@ -168,51 +154,6 @@ func (w *ShardWriter) Close() error { return nil } -const ( - maxConnections = 500 - maxRetries = 3 -) - -var errMaxConnectionsExceeded = fmt.Errorf("can not exceed max connections of %d", maxConnections) - -type connFactory struct { - nodeID uint64 - timeout time.Duration - - clientPool interface { - size() int - } - - metaClient interface { - DataNode(id uint64) (ni *meta.NodeInfo, err error) - } -} - -func (c *connFactory) dial() (net.Conn, error) { - if c.clientPool.size() > maxConnections { - return nil, errMaxConnectionsExceeded - } - - ni, err := c.metaClient.DataNode(c.nodeID) - if err != nil { - return nil, err - } - - if ni == nil { - return nil, fmt.Errorf("node %d does not exist", c.nodeID) - } - - conn, err := net.DialTimeout("tcp", ni.TCPHost, c.timeout) - if err != nil { - return nil, err - } - - // Write a marker byte for cluster messages. - _, err = conn.Write([]byte{MuxHeader}) - if err != nil { - conn.Close() - return nil, err - } - - return conn, nil +func (w *ShardWriter) Stats() []StatEntity { + return w.pool.Stat() } diff --git a/docs/architecture.png b/docs/architecture.png index 9ff5191..9822689 100644 Binary files a/docs/architecture.png and b/docs/architecture.png differ diff --git a/go.mod b/go.mod index d691a0a..186c881 100644 --- a/go.mod +++ b/go.mod @@ -1,41 +1,34 @@ module github.com/angopher/chronus require ( - collectd.org v0.3.0 // indirect - github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7 // indirect github.com/BurntSushi/toml v0.3.1 - github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40 // indirect - github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd // indirect - github.com/coreos/etcd v3.3.12+incompatible - github.com/dgraph-io/badger v1.5.5-0.20181126210712-49a49e321746 - github.com/dgraph-io/dgo v0.0.0-20190201214300-d5a1729ba705 // indirect - github.com/dgraph-io/dgraph v1.0.11 - github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f // indirect - github.com/gogo/protobuf v1.2.1 - github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db - github.com/google/go-cmp v0.2.0 - github.com/influxdata/influxdb v1.7.4 - github.com/influxdata/influxql v0.0.0-20180925231337-1cbfca8e56b6 - github.com/influxdata/platform v0.0.0-20190117200541-d500d3cf5589 - github.com/influxdata/roaring v0.4.12 // indirect + github.com/dgraph-io/badger/v2 v2.2007.2 + github.com/dgraph-io/dgraph v1.2.7 + github.com/dustin/go-humanize v1.0.0 + github.com/fatih/color v1.9.0 + github.com/go-errors/errors v1.1.1 + github.com/gogo/protobuf v1.3.1 + github.com/golang/snappy v0.0.2 + github.com/google/go-cmp v0.5.2 + github.com/influxdata/influxdb v1.8.3 + github.com/influxdata/influxql v1.1.1-0.20200828144457-65d3ef77d385 github.com/influxdata/usage-client v0.0.0-20160829180054-6d3895376368 - github.com/klauspost/compress v1.4.1 // indirect - github.com/klauspost/cpuid v1.2.0 // indirect - github.com/klauspost/pgzip v1.2.1 - github.com/pkg/errors v0.8.0 - github.com/pkg/profile v1.2.1 // indirect - github.com/spf13/cobra v0.0.3 - github.com/xlab/treeprint v0.0.0-20181112141820-a009c3971eca // indirect - go.opencensus.io v0.19.0 // indirect - go.uber.org/zap v1.9.1 - golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9 - golang.org/x/net v0.0.0-20181217023233-e147a9138326 - golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4 // indirect - golang.org/x/text v0.3.0 - golang.org/x/time v0.0.0-20181108054448-85acf8d2951c // indirect - gopkg.in/fatih/pool.v2 v2.0.0 - labix.org/v2/mgo v0.0.0-20140701140051-000000000287 // indirect - launchpad.net/gocheck v0.0.0-20140225173054-000000000087 // indirect + github.com/jsternberg/zap-logfmt v1.2.0 + github.com/klauspost/compress v1.11.1 // indirect + github.com/klauspost/pgzip v1.2.5 // indirect + github.com/kr/pretty v0.2.0 // indirect + github.com/pkg/errors v0.9.1 + github.com/stretchr/testify v1.6.1 // test + github.com/urfave/cli/v2 v2.2.0 + github.com/willf/bitset v1.1.11 // indirect + github.com/xlab/treeprint v1.0.0 // indirect + go.etcd.io/etcd v0.0.0-20200824191128-ae9734ed278b // actual it's v3.4.13. we need it because of nasty dependency issue. + go.uber.org/zap v1.16.0 + golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0 + golang.org/x/net v0.0.0-20201010224723-4f7140c49acb + golang.org/x/text v0.3.3 + golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e + gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) -go 1.13 +go 1.14 diff --git a/go.sum b/go.sum index 4e9ab42..0453325 100644 --- a/go.sum +++ b/go.sum @@ -1,401 +1,755 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.43.0/go.mod h1:BOSR3VbTLkk6FDC/TcffxP4NF/FFBGA5ku+jvKOP7pg= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.51.0 h1:PvKAVQWCtlGUSlZkGW3QLelKaWq7KYv/MW1EboG8bfM= +cloud.google.com/go v0.51.0/go.mod h1:hWtGJ6gnXH+KgDv+V0zFGDvpi07n3z8ZNj3T1RW0Gcw= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0 h1:sAbMqjY1PEQKZBWfbu6Y6bsupJ9c4QdHnzg/VvYTLcE= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigtable v1.2.0 h1:F4cCmA4nuV84V5zYQ3MKY+M1Cw1avHDuf3S/LcZPA9c= +cloud.google.com/go/bigtable v1.2.0/go.mod h1:JcVAOl45lrTmQfLj7T6TxyMzIN/3FGGcFm+2xVAli2o= +cloud.google.com/go/datastore v1.0.0 h1:Kt+gOPPp2LEPWp8CSfxhsM8ik9CcyE/gYu+0r+RnZvM= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0 h1:9/vpR43S4aJaROxqQHQ3nH9lfyKKV0dC3vOmnw8ebQQ= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0 h1:RPUcBvDeYgQFMfQu1eBMq6piD1SXmLH+vK3qjewZPus= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= collectd.org v0.3.0 h1:iNBHGw1VvPJxH2B6RiFWFZ+vsjo1lCdRszBeOuwGi00= collectd.org v0.3.0/go.mod h1:A/8DzQBkF6abtvrT2j/AU/4tiBgJWYyh0y/oB/4MlWE= -git.apache.org/thrift.git v0.0.0-20181218151757-9b75e4fe745a/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= -github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7 h1:PqzgE6kAMi81xWQA2QIVxjWkFHptGgC547vchpUbtFo= -github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= -github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8= +contrib.go.opencensus.io/exporter/jaeger v0.1.0 h1:WNc9HbA38xEQmsI40Tjd/MNU/g8byN2Of7lwIjv0Jdc= +contrib.go.opencensus.io/exporter/jaeger v0.1.0/go.mod h1:VYianECmuFPwU37O699Vc1GOcy+y8kOsfaxHRImmjbA= +contrib.go.opencensus.io/exporter/prometheus v0.1.0 h1:SByaIoWwNgMdPSgl5sMqM2KDE5H/ukPWBRo314xiDvg= +contrib.go.opencensus.io/exporter/prometheus v0.1.0/go.mod h1:cGFniUXGZlKRjzOyuZJ6mgB+PgBcCIa79kEKR8YCW+A= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/99designs/gqlgen v0.10.1/go.mod h1:IviubpnyI4gbBcj8IcxSSc/Q/+af5riwCmJmwF0uaPE= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/DataDog/datadog-go v0.0.0-20180822151419-281ae9f2d895/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= -github.com/Jeffail/gabs v1.1.1/go.mod h1:6xMvQMK4k33lb7GUUpaAPh6nKMmemQeg5d4gn7/bOXc= -github.com/Masterminds/semver v1.4.2/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= -github.com/Masterminds/sprig v2.16.0+incompatible/go.mod h1:y6hNFY5UBTIWBxnzTeuNhlNS5hqE0NB0E6fgfo2Br3o= -github.com/Microsoft/go-winio v0.4.11/go.mod h1:VhR8bwka0BXejwEJY73c50VrPtXAaKcyvVC4A4RozmA= -github.com/NYTimes/gziphandler v1.0.1/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/DATA-DOG/go-sqlmock v1.3.2/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= +github.com/DATA-DOG/go-sqlmock v1.3.3 h1:CWUqKXe0s8A2z6qCgkP4Kru7wC11YoAnoupUKFDnH08= +github.com/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= +github.com/DataDog/datadog-go v0.0.0-20190425163447-40bafcb5f6c1 h1:fSu93OUqfEkoQJBkTsxFB1e0oESqabS45iRX880e7Xw= +github.com/DataDog/datadog-go v0.0.0-20190425163447-40bafcb5f6c1/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= +github.com/DataDog/opencensus-go-exporter-datadog v0.0.0-20190503082300-0f32ad59ab08 h1:5btKvK+N+FpW0EEgvxq7LWcUEwIRLsL4IwIo0u+Qlhs= +github.com/DataDog/opencensus-go-exporter-datadog v0.0.0-20190503082300-0f32ad59ab08/go.mod h1:gMGUEe16aZh0QN941HgDjwrdjU4iTthPoz2/AtDRADE= +github.com/DataDog/zstd v1.4.1/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= +github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= +github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5/go.mod h1:lmUJ/7eu/Q8D7ML55dXQrVaamCz2vxCfdQBasLZfHKk= +github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/RoaringBitmap/roaring v0.4.16 h1:NholfewybRLOwACgfqfzn/N5xa6keKNs4fP00t0cwLo= -github.com/RoaringBitmap/roaring v0.4.16/go.mod h1:8khRDP4HmeXns4xIj9oGrKSz7XTQiJx2zgh7AcNke4w= -github.com/SAP/go-hdb v0.13.1/go.mod h1:etBT+FAi1t5k3K3tf5vQTnosgYmhDkRi8jEnQqCnxF0= -github.com/SermoDigital/jose v0.9.1/go.mod h1:ARgCUhI1MHQH+ONky/PAtmVHQrP5JlGY0F3poXOp/fA= -github.com/alcortesm/tgz v0.0.0-20161220082320-9c5fe88206d7/go.mod h1:6zEj6s6u/ghQa61ZWa/C2Aw3RkjiTBOix7dkqa1VLIs= -github.com/alecthomas/kingpin v2.2.6+incompatible/go.mod h1:59OFYbFVLKQKq+mqrL6Rw5bR0c3ACQaawgXx0QYndlE= +github.com/OneOfOne/xxhash v1.2.5 h1:zl/OfRA6nftbBK9qTohYBJ5xvw6C/oNKizR7cZGl3cI= +github.com/OneOfOne/xxhash v1.2.5/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdIIOT9Um7Q= +github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= +github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/agnivade/levenshtein v1.0.1/go.mod h1:CURSv5d9Uaml+FovSIICkLbAUZ9S4RqaHDIsdSBg7lM= +github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc h1:cAKDfWh5VpdgMhJosfJnn5/FoN2SRZ4p7fJNX58YPaU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf h1:qet1QNfXsQxTZqLG4oE62mJzwPIB8+Tee4RNCL9ulrY= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883 h1:bvNMNQO63//z+xNgfBlViaCIJKLlCJ6/fmUseuG0wVQ= github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883/go.mod h1:rCTlJbsFo29Kk6CurOXKm700vrz8f0KW0JNfpkRJY/8= -github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c= -github.com/aokoli/goutils v1.0.1/go.mod h1:SijmP0QR8LtwsmDs8Yii5Z/S4trXFGFC2oO5g9DP+DQ= -github.com/apache/arrow/go/arrow v0.0.0-20181031164735-a56c009257a7/go.mod h1:GjvccvtI06FGFvRU1In/maF7tKp3h7GBV9Sexo5rNPM= -github.com/apache/arrow/go/arrow v0.0.0-20181217213538-e9ed591db9cb h1:p6xQwsjxRtuIrUDjGAFuro04BO0GNJ9V2troYRY8kmQ= -github.com/apache/arrow/go/arrow v0.0.0-20181217213538-e9ed591db9cb/go.mod h1:GjvccvtI06FGFvRU1In/maF7tKp3h7GBV9Sexo5rNPM= -github.com/apex/log v1.1.0/go.mod h1:yA770aXIDQrhVOIGurT/pVdfCpSq1GQV/auzMN5fzvY= -github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/asaskevich/govalidator v0.0.0-20180720115003-f9ffefc3facf/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY= -github.com/aws/aws-sdk-go v1.15.59/go.mod h1:E3/ieXAlvM0XWO57iftYVDLLvQ824smPP3ATZkfNZeM= -github.com/aws/aws-sdk-go v1.15.64/go.mod h1:E3/ieXAlvM0XWO57iftYVDLLvQ824smPP3ATZkfNZeM= -github.com/benbjohnson/tmpl v1.0.0/go.mod h1:igT620JFIi44B6awvU9IsDhR77IXWtFigTLil/RPdps= +github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db h1:nxAtV4VajJDhKysp2kdcJZsq8Ss1xSA0vZTkVHHJd0E= +github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db/go.mod h1:VTxUBvSJ3s3eHAg65PNgrsn5BtqCRPdmyXh6rAfdxN0= +github.com/apache/thrift v0.12.0 h1:pODnxUFNcjP9UTLZGTdeh+j16A8lJbRvD3rOtrk/7bs= +github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= +github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932/go.mod h1:NOuUCSz6Q9T7+igc/hlvDOUdtWKryOrtFyIVABv/p7k= -github.com/blakesmith/ar v0.0.0-20150311145944-8bd4349a67f2/go.mod h1:PkYb9DJNAwrSvRx5DYA+gUcOIgTGVMNkfSCbZM8cWpI= -github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= +github.com/beorn7/perks v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0= +github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= +github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/blevesearch/bleve v0.0.0-20181114232033-e1f5e6cdcd76/go.mod h1:Y2lmIkzV6mcNfAnAdOd+ZxHkHchhBfU/xroGIp61wfw= +github.com/blevesearch/go-porterstemmer v1.0.2/go.mod h1:haWQqFT3RdOGz7PJuM3or/pWNJS1pKkoZJWCkWu0DVA= +github.com/blevesearch/segment v0.0.0-20160915185041-762005e7a34f/go.mod h1:IInt5XRvpiGE09KOk9mmCMLjHhydIhNPKPPFLFBB7L8= +github.com/blevesearch/snowballstem v0.0.0-20180110192139-26b06a2c243d/go.mod h1:cdytUvf6FKWA9NpXJihYdZq8TN2AiQ5HOS0UZUz0C9g= github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40 h1:y4B3+GPxKlrigF1ha5FFErxK+sr6sWxQovRMzwMhejo= github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40/go.mod h1:8rLXio+WjiTceGBHIoTvn60HIbs7Hm7bcHjyrSqYB9c= +github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= -github.com/bouk/httprouter v0.0.0-20160817010721-ee8b3818a7f5/go.mod h1:CDReaxg1cmLrtcasZy43l4EYPAknXLiQSrb7tLw5zXM= +github.com/c-bata/go-prompt v0.2.2 h1:uyKRz6Z6DUyj49QVijyM339UJV9yhbr70gESwbNU3e0= github.com/c-bata/go-prompt v0.2.2/go.mod h1:VzqtzE2ksDBcdln8G7mk2RX9QyGjH+OVqOCSiVIqS34= -github.com/caarlos0/ctrlc v1.0.0/go.mod h1:CdXpj4rmq0q/1Eb44M9zi2nKB0QraNKuRGYGrrHhcQw= -github.com/campoy/unique v0.0.0-20180121183637-88950e537e7e/go.mod h1:9IOqJGCPMSc6E5ydlp5NIonxObaeu/Iub/X03EKPVYo= -github.com/cenkalti/backoff v2.0.0+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= +github.com/cenkalti/backoff v2.1.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/circonus-labs/circonus-gometrics v2.2.5+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= -github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= +github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= -github.com/containerd/continuity v0.0.0-20181027224239-bea7585dbfac/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= -github.com/coreos/bbolt v1.3.1-coreos.6/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= -github.com/coreos/etcd v3.3.12+incompatible h1:pAWNwdf7QiT1zfaWyqCtNZQWCLByQyA3JrSQyuYAqnQ= -github.com/coreos/etcd v3.3.12+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/containerd/continuity v0.0.0-20181203112020-004b46473808/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= +github.com/coreos/etcd v3.3.10+incompatible h1:jFneRYjIvLMLhDLCzuTuU4rSJUjRplcJQ7pD7MnhC04= +github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= +github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= +github.com/cpuguy83/go-md2man v1.0.10 h1:BSKMNlYxDvnunlTymqtgONjNnaRV1sTpcovwwjF22jk= +github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= +github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d h1:U+s90UTSYgptZMwQh2aRr3LuazLJIa+Pg3Kc1ylSYVY= +github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= +github.com/d4l3k/messagediff v1.2.1/go.mod h1:Oozbb1TVXFac9FtSIxHBMnBCq2qeH/2KkEQxENCrlLo= +github.com/dave/jennifer v1.2.0/go.mod h1:fIb+770HOpJ2fmN9EPPKOqm1vMGhB+TwXKMZhrIygKg= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/denisenkom/go-mssqldb v0.0.0-20181014144952-4e0d7dc8888f/go.mod h1:xN/JuLBIz4bjkxNmByTiV1IbhfnYb6oo99phBn4Eqhc= -github.com/dgraph-io/badger v1.5.5-0.20181126210712-49a49e321746 h1:cd1s0THXu7KWnykg7V07VSiTde/g38d8PJBTXBaq0mk= -github.com/dgraph-io/badger v1.5.5-0.20181126210712-49a49e321746/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= -github.com/dgraph-io/dgo v0.0.0-20190201214300-d5a1729ba705 h1:f2knGDNK8Q5y9D8rKdACl7qviC4jonzug90y4TNaRxU= -github.com/dgraph-io/dgo v0.0.0-20190201214300-d5a1729ba705/go.mod h1:/3OtwVvGnHiyXL9bWTjZaHQk4cp8rakX6jVBFoykWaA= -github.com/dgraph-io/dgraph v1.0.11 h1:An8I4+pG7e6n3evXXZsB4E6f3PUc9+0ynBRVCivb8f4= -github.com/dgraph-io/dgraph v1.0.11/go.mod h1:Qx1vy3ocQ5/Q66iKt6KcFADD3VUncMHuVaNF0sgr9js= +github.com/dgraph-io/badger/v2 v2.2007.2-0.20200827131741-d5a25b83fbf4 h1:DUDFTVgqZysKplH39/ya0aI4+zGm91L9QttXgITT2YE= +github.com/dgraph-io/badger/v2 v2.2007.2-0.20200827131741-d5a25b83fbf4/go.mod h1:26P/7fbL4kUZVEVKLAKXkBXKOydDmM2p1e+NhhnBCAE= +github.com/dgraph-io/badger/v2 v2.2007.2 h1:EjjK0KqwaFMlPin1ajhP943VPENHJdEz1KLIegjaI3k= +github.com/dgraph-io/badger/v2 v2.2007.2/go.mod h1:26P/7fbL4kUZVEVKLAKXkBXKOydDmM2p1e+NhhnBCAE= +github.com/dgraph-io/dgo/v2 v2.1.1-0.20191127085444-c7a02678e8a6 h1:5leDFqGys055YO3TbghBhk/QdRPEwyLPdgsSJfiR20I= +github.com/dgraph-io/dgo/v2 v2.1.1-0.20191127085444-c7a02678e8a6/go.mod h1:LJCkLxm5fUMcU+yb8gHFjHt7ChgNuz3YnQQ6MQkmscI= +github.com/dgraph-io/dgraph v1.2.7 h1:umWwQ1wPpRJqhOx52IFJ4ixCU5//anPK6VbT+N67L5Y= +github.com/dgraph-io/dgraph v1.2.7/go.mod h1:q0S+BGN+D8UUDHbK+oECqCU27NIzWwNLJ5vDw4+oAhs= +github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= +github.com/dgraph-io/ristretto v0.0.4-0.20200904131139-4dec2770af66 h1:ectpJv2tGhTudyk0JhqE/53o/ObH30u5yt/yThsAn3I= +github.com/dgraph-io/ristretto v0.0.4-0.20200904131139-4dec2770af66/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-bitstream v0.0.0-20180413035011-3522498ce2c8 h1:akOQj8IVgoeFfBTzGOEQakCYshWD6RNo1M5pivFXt70= github.com/dgryski/go-bitstream v0.0.0-20180413035011-3522498ce2c8/go.mod h1:VMaSuZ+SZcx/wljOQKvp5srsbCiKDEb6K2wC4+PiBmQ= -github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f h1:dDxpBYafY/GYpcl+LS4Bn3ziLPuEdGRkRjYAbSlWxSA= -github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/docker/distribution v2.6.2+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= -github.com/docker/docker v0.0.0-20180422163414-57142e89befe/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= +github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dgryski/go-groupvarint v0.0.0-20190318181831-5ce5df8ca4e1/go.mod h1:MlkUQveSLEDbIgq2r1e++tSf0zfzU9mQpa9Qkczl+9Y= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= github.com/docker/go-units v0.3.3/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= -github.com/duosecurity/duo_api_golang v0.0.0-20181024123116-92fea9203dbc/go.mod h1:UqXY1lYT/ERa4OEAywUqdok1T4RCRdArkhic1Opuavo= +github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/elazarl/go-bindata-assetfs v1.0.0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= -github.com/emirpasic/gods v1.9.0/go.mod h1:YfzfFFoVP/catgzJb4IKIqXjX78Ha8FMSDh3ymbK86o= +github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/eclipse/paho.mqtt.golang v1.2.0 h1:1F8mhG9+aO5/xpdtFkW4SxOJB67ukuDC3t2y2qayIX0= +github.com/eclipse/paho.mqtt.golang v1.2.0/go.mod h1:H9keYFcgq3Qr5OUJm/JZI/i6U7joQ8SYLhZwfeOo6Ts= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= -github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= -github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= +github.com/fatih/color v1.9.0 h1:8xPHl4/q1VyqGIPif1F+1V3Y3lSmrq01EabUW3CoW5s= +github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= +github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/getkin/kin-openapi v0.1.0/go.mod h1:+0ZtELZf+SlWH8ZdA/IeFb3L/PKOKJx8eGxAlUZ/sOU= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0= +github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32/go.mod h1:GIjDIg/heH5DOkXY3YJ/wNhfHsQHoXGjl8G8amsYQ1I= github.com/glycerine/go-unsnap-stream v0.0.0-20180323001048-9f0cb55181dd h1:r04MMPyLHj/QwZuMJ5+7tJcBr1AQjpiAK/rZWRrQT7o= github.com/glycerine/go-unsnap-stream v0.0.0-20180323001048-9f0cb55181dd/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= -github.com/glycerine/goconvey v0.0.0-20180728074245-46e3a41ad493/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= -github.com/go-ldap/ldap v2.5.1+incompatible/go.mod h1:qfd9rJvER9Q0/D/Sqn1DfHRoBp40uXYvFoEVrNEPqRc= -github.com/go-sql-driver/mysql v1.4.0 h1:7LxgVwFb2hIQtMm87NdgAVfXjnt4OePseqT1tKx+opk= -github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-test/deep v1.0.1/go.mod h1:wGDj63lr65AM2AQyKZd/NYHGb0R+1RLqB8NKt3aSFNA= -github.com/gocql/gocql v0.0.0-20181117210152-33c0e89ca93a/go.mod h1:4Fw1eo5iaEhDUs8XyuhSVCVy52Jq3L+/3GJgYkwc+/0= +github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31 h1:gclg6gY70GLy3PbkQ1AERPfmLMMagS60DKF78eWwLn8= +github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= +github.com/go-chi/chi v3.3.2+incompatible/go.mod h1:eB3wogJHnLi3x/kFX2A+IbTBlXxmMeXJVKy9tTv1XzQ= +github.com/go-errors/errors v1.1.1 h1:ljK/pL5ltg3qoN+OtN6yCv9HWSfMwxSx90GJCZQxYNg= +github.com/go-errors/errors v1.1.1/go.mod h1:psDX2osz5VnTOnFWbDeWwS7yejl+uV3FEWEp4lssFEs= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-ini/ini v1.39.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-sql-driver/mysql v0.0.0-20190330032241-c0f6b444ad8f/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-sql-driver/mysql v1.4.1 h1:g24URVg0OFbNUTx9qqY1IRZ9D9z3iPyi5zKhQZpNwpA= +github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gofrs/uuid v3.3.0+incompatible h1:8K4tyRfvU1CYPgJsveYFQMhpFd/wXNM7iK6rR7UHz84= +github.com/gofrs/uuid v3.3.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= +github.com/gogo/protobuf v1.0.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE= +github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= -github.com/golang/gddo v0.0.0-20181116215533-9bd4a3295021/go.mod h1:xEhNfoBDX1hzLm2Nf80qUvZ2sVwoMZ8d6IE2SrsQfh4= +github.com/gogo/protobuf v1.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= +github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= +github.com/golang/geo v0.0.0-20170810003146-31fb0106dc4a/go.mod h1:vgWZ7cu0fq0KY3PpEHsocXOWJpRtkcbKemU4IUw0M60= +github.com/golang/geo v0.0.0-20190916061304-5b978397cfec h1:lJwO/92dFXWeXOZdoGXgptLmNLwynMSHUmU6besqtiw= +github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E= +github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7 h1:5ZkaAPbicIKTF2I64qf5Fh8Aa83Q/dnOafMYV0OMwjA= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/snappy v0.0.0-20170215233205-553a64147049/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.5 h1:F768QJ1E9tib+q5Sc8MkdJi1RxLTbRcTf8LJV56aRls= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db h1:woRePGFeVFfLKN/pOkfl+p/TAqKOfFu+7KPlMVpok/w= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.2 h1:aeE13tS0IiQgFjYdoL8qN3K1N2bXXtI6Vi51/y7BpMw= +github.com/golang/snappy v0.0.2/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c h1:964Od4U6p2jUkFxvCydnIczKteheJEzHRToSGK3Bnlw= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/codesearch v1.0.0/go.mod h1:qCnXDFnak/trCmLaE50kgPte3AX9jSeruZexWEOivi0= +github.com/google/flatbuffers v1.11.0 h1:O7CEyB8Cb3/DmtxODGtLHcEvpr81Jm5qLg/hsHnxA2A= +github.com/google/flatbuffers v1.11.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/go-cmp v0.2.0 h1:+dTQ8DZQJz0Mb/HjFlkptS1FeQ4cWSnN941F8aEG4SQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= -github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+umXrS52loVEgC2AApnigrVQ= -github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= +github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/goreleaser/goreleaser v0.94.0/go.mod h1:OjbYR2NhOI6AEUWCowMSBzo9nP1aRif3sYtx+rhp+Zo= -github.com/goreleaser/nfpm v0.9.7/go.mod h1:F2yzin6cBAL9gb+mSiReuXdsfTrOQwDMsuSpULof+y4= -github.com/gotestyourself/gotestyourself v2.2.0+incompatible/go.mod h1:zZKM6oeNM8k+FRljX1mnzVYeS8wiGgQyvST1/GafPbY= -github.com/grpc-ecosystem/grpc-gateway v1.6.2/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= -github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed/go.mod h1:tMWxXQ9wFIaZeTI9F+hmhFiGpFmhOHzyShyFUhRm0H4= -github.com/hashicorp/consul v1.4.0/go.mod h1:mFrjN1mfidgJfYP1xrJCF+AfRhr6Eaqhb2+sfyn/OOI= -github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= -github.com/hashicorp/go-hclog v0.0.0-20180709165350-ff2cf002a8dd/go.mod h1:9bjs9uLqI8l75knNv3lV1kA55veR+WUPSiKIWcQHudI= -github.com/hashicorp/go-hclog v0.0.0-20181001195459-61d530d6c27f/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ= -github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-memdb v0.0.0-20181108192425-032f93b25bec/go.mod h1:kbfItVoBJwCfKXDXN4YoAXjxcFVZ7MRrJzyTX6H4giE= -github.com/hashicorp/go-msgpack v0.0.0-20150518234257-fa3f63826f7c/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= -github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= -github.com/hashicorp/go-plugin v0.0.0-20181030172320-54b6ff97d818/go.mod h1:Ft7ju2vWzhO0ETMKUVo12XmXmII6eSUS4rsPTkY/siA= -github.com/hashicorp/go-retryablehttp v0.5.0/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= -github.com/hashicorp/go-rootcerts v0.0.0-20160503143440-6bb64b370b90/go.mod h1:o4zcYY1e0GEZI6eSEr+43QDYmuGglw1qSO6qdHUHCgg= -github.com/hashicorp/go-sockaddr v0.0.0-20180320115054-6d291a969b86/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= -github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-version v1.0.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/gorilla/context v0.0.0-20160226214623-1ea25387ff6f/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= +github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= +github.com/gorilla/mux v1.6.1/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= +github.com/gorilla/websocket v1.2.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= +github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= +github.com/grpc-ecosystem/grpc-gateway v1.4.1/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= +github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= +github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hashicorp/memberlist v0.1.0/go.mod h1:ncdBp14cuox2iFOq3kDiquKU6fqsTBc3W6JvZwjxxsE= -github.com/hashicorp/raft v1.0.0/go.mod h1:DVSAWItjLjTOkVbSpWQ0j0kUADIvDaCtBxIcbNAQLkI= -github.com/hashicorp/serf v0.8.1/go.mod h1:h/Ru6tmZazX7WO/GDmwdpS975F019L4t5ng5IgwbNrE= -github.com/hashicorp/vault v0.11.5/go.mod h1:KfSyffbKxoVyspOdlaGVjIuwLobi07qD1bAbosPMpP0= -github.com/hashicorp/vault-plugin-secrets-kv v0.0.0-20181106190520-2236f141171e/go.mod h1:VJHHT2SC1tAPrfENQeBhLlb5FbZoKZM+oC/ROmEftz0= -github.com/hashicorp/yamux v0.0.0-20180604194846-3520598351bb/go.mod h1:+NfK9FKeTrX5uv1uIXGdwYDTeHna2qgaIlx54MXqjAM= -github.com/hashicorp/yamux v0.0.0-20181012175058-2f1d1f20f75d/go.mod h1:+NfK9FKeTrX5uv1uIXGdwYDTeHna2qgaIlx54MXqjAM= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/huandu/xstrings v1.0.0/go.mod h1:4qWG/gcEcfX4z/mBDHJ++3ReCw9ibxbsNJbcucJdbSo= -github.com/imdario/mergo v0.3.4/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA= -github.com/imdario/mergo v0.3.6/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/influxdata/flux v0.13.0 h1:iWNTxfR8m8LFYNAjSXAYMdxMKHqsBxd3xHlZ7rpl1KA= -github.com/influxdata/flux v0.13.0/go.mod h1:81jeDcHVn1rN5uj9aQ81S72Q8ol8If7N0zM0G8TnxTE= -github.com/influxdata/influxdb v1.7.4 h1:Ufqfn5xFixUXXj5Fgmhfa9RSke2R2AOvUOXfxgp9SCA= -github.com/influxdata/influxdb v1.7.4/go.mod h1:qZna6X/4elxqT3yI9iZYdZrWWdeFOOprn86kgg4+IzY= -github.com/influxdata/influxql v0.0.0-20180925231337-1cbfca8e56b6 h1:CFx+pP90q/qg3spoiZjf8donE4WpAdjeJfPOcoNqkWo= -github.com/influxdata/influxql v0.0.0-20180925231337-1cbfca8e56b6/go.mod h1:KpVI7okXjK6PRi3Z5B+mtKZli+R1DnZgb3N+tzevNgo= +github.com/influxdata/flux v0.65.1 h1:77BcVUCzvN5HMm8+j9PRBQ4iZcu98Dl4Y9rf+J5vhnc= +github.com/influxdata/flux v0.65.1/go.mod h1:J754/zds0vvpfwuq7Gc2wRdVwEodfpCFM7mYlOw2LqY= +github.com/influxdata/influxdb v1.8.3 h1:WEypI1BQFTT4teLM+1qkEcvUi0dAvopAI/ir0vAiBg8= +github.com/influxdata/influxdb v1.8.3/go.mod h1:JugdFhsvvI8gadxOI6noqNeeBHvWNTbfYGtiAn+2jhI= +github.com/influxdata/influxql v1.1.1-0.20200828144457-65d3ef77d385 h1:ED4e5Cc3z5vSN2Tz2GkOHN7vs4Sxe2yds6CXvDnvZFE= +github.com/influxdata/influxql v1.1.1-0.20200828144457-65d3ef77d385/go.mod h1:gHp9y86a/pxhjJ+zMjNXiQAA197Xk9wLxaz+fGG+kWk= github.com/influxdata/line-protocol v0.0.0-20180522152040-32c6aa80de5e h1:/o3vQtpWJhvnIbXley4/jwzzqNeigJK9z+LZcJZ9zfM= github.com/influxdata/line-protocol v0.0.0-20180522152040-32c6aa80de5e/go.mod h1:4kt73NQhadE3daL3WhR5EJ/J2ocX0PZzwxQ0gXJ7oFE= -github.com/influxdata/platform v0.0.0-20190117200541-d500d3cf5589 h1:oN2MMxbnMD/XIlyXbSczQqN5vcrCMFuRsiQafSt2c1E= -github.com/influxdata/platform v0.0.0-20190117200541-d500d3cf5589/go.mod h1:YVhys+JOY4wmXtJvdtkzLhS2K/r/px/vPc+EAddK+pg= -github.com/influxdata/roaring v0.4.12 h1:3DzTjKHcXFs4P3D7xRLpCqVrfK6eFRQT0c8BG99M3Ms= -github.com/influxdata/roaring v0.4.12/go.mod h1:bSgUQ7q5ZLSO+bKBGqJiCBGAl+9DxyW63zLTujjUlOE= +github.com/influxdata/promql/v2 v2.12.0/go.mod h1:fxOPu+DY0bqCTCECchSRtWfc+0X19ybifQhZoQNF5D8= +github.com/influxdata/roaring v0.4.13-0.20180809181101-fc520f41fab6 h1:UzJnB7VRL4PSkUJHwsyzseGOmrO/r4yA+AuxGJxiZmA= +github.com/influxdata/roaring v0.4.13-0.20180809181101-fc520f41fab6/go.mod h1:bSgUQ7q5ZLSO+bKBGqJiCBGAl+9DxyW63zLTujjUlOE= github.com/influxdata/tdigest v0.0.0-20181121200506-bf2b5ad3c0a9 h1:MHTrDWmQpHq/hkq+7cw9oYAt2PqUw52TZazRA0N7PGE= github.com/influxdata/tdigest v0.0.0-20181121200506-bf2b5ad3c0a9/go.mod h1:Js0mqiSBE6Ffsg94weZZ2c+v/ciT8QRHFOap7EKDrR0= github.com/influxdata/usage-client v0.0.0-20160829180054-6d3895376368 h1:+TUUmaFa4YD1Q+7bH9o5NCHQGPMqZCYJiNW6lIIS9z4= github.com/influxdata/usage-client v0.0.0-20160829180054-6d3895376368/go.mod h1:Wbbw6tYNvwa5dlB6304Sd+82Z3f7PmVZHVKU637d4po= -github.com/jbenet/go-context v0.0.0-20150711004518-d14ea06fba99/go.mod h1:1lJo3i6rXxKeerYnT8Nvf0QmHCRC1n8sfWVwXF2Frvo= -github.com/jefferai/jsonx v0.0.0-20160721235117-9cc31c3135ee/go.mod h1:N0t2vlmpe8nyZB5ouIbJQPDSR+mH6oe7xHB9VZHSUzM= -github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jmespath/go-jmespath v0.0.0-20160202185014-0b12d6b521d8/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= -github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= +github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1 h1:6QPYqodiu3GuPL+7mfx+NwDdp2eTkp9IfEUpgAwUN0o= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/jsternberg/zap-logfmt v1.0.0/go.mod h1:uvPs/4X51zdkcm5jXl5SYoN+4RK21K8mysFmDaM/h+o= github.com/jsternberg/zap-logfmt v1.2.0 h1:1v+PK4/B48cy8cfQbxL4FmmNZrjnIMr2BsnyEmXqv2o= github.com/jsternberg/zap-logfmt v1.2.0/go.mod h1:kz+1CUmCutPWABnNkOu9hOHKdT2q3TDYCcsFy9hpqb0= -github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jwilder/encoding v0.0.0-20170811194829-b4e1701a28ef h1:2jNeR4YUziVtswNP9sEFAI913cVrzH85T+8Q6LpYbT0= github.com/jwilder/encoding v0.0.0-20170811194829-b4e1701a28ef/go.mod h1:Ct9fl0F6iIOGgxJ5npU/IUOhOhqlVrGjyIZc8/MagT0= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= -github.com/kevinburke/go-bindata v3.11.0+incompatible/go.mod h1:/pEEZ72flUW2p0yi30bslSp9YqD9pysLxunQDdb2CPM= -github.com/kevinburke/ssh_config v0.0.0-20180830205328-81db2a75821e/go.mod h1:CT57kijsi8u/K/BOFA39wgDQJ9CxiF4nAY/ojJ6r6mM= -github.com/keybase/go-crypto v0.0.0-20181031135447-f919bfda4fc1/go.mod h1:ghbZscTyKdM07+Fw3KSi0hcJm+AlEUWj8QLlPtijN/M= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.4.1/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/cpuid v1.2.0/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= -github.com/klauspost/pgzip v1.2.1/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= +github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.11.1 h1:bPb7nMRdOZYDrpPMTA3EInUQrdgoBinqUuSwlGdKDdE= +github.com/klauspost/compress v1.11.1/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= +github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPRLkC4NPOvfYeR5KNOrY6TD+/sAC3HXPZgDYg= +github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= +github.com/klauspost/pgzip v1.2.5 h1:qnWYvvKqedOF2ulHpMG72XQol4ILEJ8k2wwRl/Km8oE= +github.com/klauspost/pgzip v1.2.5/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= +github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pty v1.0.0/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/lib/pq v1.0.0 h1:X5PMW56eZitiTeO7tKzZxFCSpbFZJtkMMooicw2us9A= github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/magiconair/properties v1.8.0 h1:LLgXmsheXeRoUOBOjtwPQCWIYqM/LU1ayDtDePerRcY= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= +github.com/mattn/go-colorable v0.0.9 h1:UVL0vNpWh04HeJXV0KLcaT7r06gOH2l4OW6ddYRUIY4= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +github.com/mattn/go-colorable v0.1.4 h1:snbPLB8fVfU9iwbbo30TPtbLRzwWu6aJS6Xh4eaaviA= +github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-isatty v0.0.4 h1:bnP0vzxcAdeI1zdubAl5PjU6zsERjGZb7raWodagDYs= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= +github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.11 h1:FxPOTFNqGkuDUGi3H/qkUbQO4ZiBa2brKq5r0l8TGeM= +github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= +github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= +github.com/mattn/go-runewidth v0.0.3 h1:a+kO+98RDGEfo6asOGMmpodZq4FNtnGP54yps8BzLR4= github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= +github.com/mattn/go-sqlite3 v1.11.0 h1:LDdKkqtYlom37fkvqs8rMPFKAMe8+SgjbwZ6ex1/A/Q= +github.com/mattn/go-sqlite3 v1.11.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104 h1:d8RFOZ2IiFtFWBcKEHAFYJcPTf0wY5q0exFNJZVWa1U= github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE= -github.com/mattn/go-zglob v0.0.0-20171230104132-4959821b4817/go.mod h1:9fxibJccNxU2cnpIKLRRFA7zX7qhkJIQWBb449FYHOo= -github.com/mattn/go-zglob v0.0.0-20180803001819-2ea3427bfa53/go.mod h1:9fxibJccNxU2cnpIKLRRFA7zX7qhkJIQWBb449FYHOo= +github.com/matttproud/golang_protobuf_extensions v1.0.0/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/miekg/dns v1.1.1/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/mitchellh/copystructure v1.0.0/go.mod h1:SNtv71yrdKgLRyLFxmLdkAbkKEFWgYaq1OVrnRcwhnw= -github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/go-testing-interface v0.0.0-20171004221916-a61a99592b77/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= -github.com/mitchellh/mapstructure v1.0.0/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/minio/minio-go v0.0.0-20181109183348-774475480ffe/go.mod h1:7guKYtitv8dktvNUGrhzmNlA5wrAABTQXCoesZdFQO8= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v0.0.0-20180203102830-a4e142e9c047/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/reflectwalk v1.0.0/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= -github.com/mna/pigeon v1.0.1-0.20180808201053-bb0192cfc2ae/go.mod h1:Iym28+kJVnC1hfQvv5MUtI6AiFFzvQjHcvI4RFTG/04= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/mschoch/smat v0.0.0-20160514031455-90eadee771ae h1:VeRdUYdCw49yizlSbMEn2SZ+gT+3IUKx8BqxyQdz+BY= github.com/mschoch/smat v0.0.0-20160514031455-90eadee771ae/go.mod h1:qAyveg+e4CE+eKJXWVjKXM4ck2QobLqTDytGJbLLhJg= -github.com/nats-io/gnatsd v1.3.0/go.mod h1:nqco77VO78hLCJpIcVfygDP2rPGfsEHkGTUk94uh5DQ= -github.com/nats-io/go-nats v1.6.0/go.mod h1:+t7RHT5ApZebkrQdnn6AhQJmhJJiKAvJUio1PiiCtj0= -github.com/nats-io/go-nats-streaming v0.4.0/go.mod h1:gfq4R3c9sKAINOpelo0gn/b9QDMBZnmrttcsNF+lqyo= -github.com/nats-io/nats-streaming-server v0.11.2/go.mod h1:RyqtDJZvMZO66YmyjIYdIvS69zu/wDAkyNWa8PIUa5c= -github.com/nats-io/nuid v1.0.0/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= -github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/opencontainers/go-digest v1.0.0-rc1/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s= github.com/opencontainers/image-spec v1.0.1/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0= github.com/opencontainers/runc v0.1.1/go.mod h1:qT5XzbpPznkRYVz/mWwUaVBUv2rmF59PVA73FjuZG0U= +github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.0.2 h1:3jA2P6O1F9UOrWVpwrIo17pu01KWvNWg4X946/Y5Zwg= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/openzipkin/zipkin-go v0.1.3/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= -github.com/ory/dockertest v3.3.2+incompatible/go.mod h1:1vX4m9wsvi00u5bseYwXaSnhNrne+V0E6LAcBILJdPs= -github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= -github.com/patrickmn/go-cache v2.1.0+incompatible/go.mod h1:3Qf8kWWT7OJRJbdiICTKqZju1ZixQ/KpMGzzAfe6+WQ= -github.com/pelletier/go-buffruneio v0.2.0/go.mod h1:JkE26KsDizTr40EUHkXVtNPvgGtbSNq5BcowyYOWdKo= +github.com/opentracing/opentracing-go v1.0.3-0.20180606204148-bd9c31933947 h1:oFoBvyA9Xh7MJd5dtfgocpsfjZUjh50IHPlDB0tILBs= +github.com/opentracing/opentracing-go v1.0.3-0.20180606204148-bd9c31933947/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw= +github.com/ory/dockertest v3.3.4+incompatible/go.mod h1:1vX4m9wsvi00u5bseYwXaSnhNrne+V0E6LAcBILJdPs= +github.com/paulbellamy/ratecounter v0.2.0/go.mod h1:Hfx1hDpSGoqxkVVpBi/IlYD7kChlfo5C6hzIHwPqfFE= +github.com/paulmach/go.geojson v0.0.0-20170327170536-40612a87147b/go.mod h1:YaKx1hKpWF+T2oj2lFJPsW/t1Q5e1jQI61eoQSTwpIs= github.com/pelletier/go-toml v1.2.0 h1:T5zMGML61Wp+FlcbWjRDT7yAxhJNAiPPLOFECq181zc= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= +github.com/peterh/liner v1.0.1-0.20180619022028-8c1271fcf47f/go.mod h1:xIteQHvHuaLYG9IFj6mSxM0fCKrs34IrEQUhOYuGPHc= github.com/philhofer/fwd v1.0.0 h1:UbZqGr5Y38ApvM/V/jEljVxwocdweyH+vmYvRPBnbqQ= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= +github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pkg/errors v0.8.0 h1:WdK/asTD0HN+q6hsWO3/vpuAkAr+tw6aNJNDFFf0+qw= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/profile v1.2.1 h1:F++O52m40owAmADcojzM+9gyjmMOY/T4oYJkgFDH8RE= github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= +github.com/pkg/term v0.0.0-20180730021639-bffc007b7fd5 h1:tFwafIEMf0B7NlcxV/zJ6leBIa81D3hgGSgsE5hCkOQ= github.com/pkg/term v0.0.0-20180730021639-bffc007b7fd5/go.mod h1:eCbImbZ95eXtAUIbLAuAVnBnwf83mjf6QIVH8SHYwqQ= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v0.0.0-20171201122222-661e31bf844d/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.2 h1:awm861/B8OKDd2I/6o1dy3ra4BamzKhYOiGItCeZ740= +github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM= +github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= +github.com/prometheus/client_golang v1.0.0 h1:vrDKnkGzuGvhNAL56c7DBz29ZL+KxnoR0x7enabFceM= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_model v0.0.0-20170216185247-6f3806018612/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 h1:idejC8f05m9MGOsuEi1ATq9shN03HrxNkD/luQvxCv8= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= +github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 h1:gQz4mCbXsO+nc9n1hCxHcGA3Zx3Eo+UHZoInFGUIXNM= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/common v0.0.0-20180518154759-7600349dcfe1/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.0.0-20181218105931-67670fe90761 h1:z6tvbDJ5OLJ48FFmnksv04a78maSTRBUIhkdHYV5Y98= -github.com/prometheus/common v0.0.0-20181218105931-67670fe90761/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= +github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.6.0 h1:kRhiuYSXR3+uv2IbVbZhUxK5zVD/2pp3Gd2PpvPkpEo= +github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc= +github.com/prometheus/procfs v0.0.0-20180612222113-7d6f385de8be/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a h1:9a8MnZMP0X2nLJdBg+pBmGgkJlSaKC2KaQmTCk1XDtE= github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/ryanuber/go-glob v0.0.0-20170128012129-256dc444b735/go.mod h1:807d1WSdnB0XRJzKNil9Om6lcp/3a0v4qIHxIXzX/Yc= -github.com/satori/go.uuid v1.2.0 h1:0uYX9dsZ2yD7q2RtLRtPSdGDWzjeM3TbMJP9utgA0ww= -github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= -github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= +github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.0-20190517135640-51af30a78b0e/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.0.2 h1:6LJUbpNm42llc4HRCuvApCSWB/WfhuNo9K98Q9sNGfs= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/retailnext/hllpp v1.0.1-0.20180308014038-101a6d2f8b52/go.mod h1:RDpi1RftBQPUCDRw6SmxeaREsAaRKnOclghuzp/WRzc= +github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rs/cors v1.6.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/russross/blackfriday v1.5.2 h1:HyvC0ARfnZBqnXwABFeSZHpKvJHJJfPz81GNueLj0oo= +github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/russross/blackfriday/v2 v2.0.1 h1:lPqVAte+HuHNfhJ/0LC98ESWRz8afy9tM/0RK8m9o+Q= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/segmentio/kafka-go v0.1.0 h1:IXCHG+sXPNiIR5pC/vTEItZduPKu4cnpr85YgxpxlW0= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= +github.com/segmentio/kafka-go v0.2.0 h1:HtCSf6B4gN/87yc5qTl7WsxPKQIIGXLPPM1bMCPOsoY= +github.com/segmentio/kafka-go v0.2.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= +github.com/sergi/go-diff v1.0.0 h1:Kpca3qRNrduNnOQeazBd0ysaKrUJiIuISHxogkT9RPQ= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= +github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= +github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= +github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= +github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/shurcooL/vfsgen v0.0.0-20180121065927-ffb13db8def0/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= +github.com/sirupsen/logrus v1.0.5/go.mod h1:pMByvHTf9Beacp5x1UXfOR9xyW/9antXMhjMPG0dEzc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v0.0.0-20180222194500-ef6db91d284a/go.mod h1:XDJAKZRPZ1CvBcN2aX5YOUTYGHki24fSF0Iv48Ibg0s= +github.com/smartystreets/goconvey v0.0.0-20190731233626-505e41936337/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= +github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= +github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2 h1:m8/z1t7/fwjysjQRYbP0RD+bUIF/8tJwPdEZsI83ACI= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/cast v1.2.0 h1:HHl1DSRbEQN2i8tJmtS6ViPyHx35+p51amrdsiTCrkg= -github.com/spf13/cast v1.2.0/go.mod h1:r2rcYCSwa1IExKTDiTfzaxqT2FNHs8hODu4LnUfgKEg= +github.com/spf13/cast v1.3.0 h1:oget//CVOEoFewqQxwr0Ej5yjygnqGkvggSE/gB35Q8= +github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3 h1:ZlrZ4XsMRm04Fr5pSFxBgfND2EBVa1nLpiy1stUsX/8= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= +github.com/spf13/cobra v0.0.5 h1:f0B+LkLX6DtmRH1isoNA9VTtNUK9K8xYd28JNNfOv/s= +github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/jwalterweatherman v1.0.0 h1:XHEdyB+EcvlqZamSM4ZOMGlc93t6AcsBEu9Gc1vn7yk= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.2/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.3 h1:zPAT6CGy6wXeQ7NtTnaTerfKOsV6V6F8agHXFiazDkg= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/spf13/viper v1.2.1 h1:bIcUwXqLseLF3BDAZduuNfekWG87ibtFxi59Bq+oI9M= -github.com/spf13/viper v1.2.1/go.mod h1:P4AexN0a+C9tGAnUFNwDMYYZv3pjFuvmeiMyKRaNVlI= -github.com/src-d/gcfg v1.4.0/go.mod h1:p/UMsR43ujA89BJY9duynAwIpvqEujIH/jFlfL7jWoI= -github.com/stevvooe/resumable v0.0.0-20180830230917-22b14a53ba50/go.mod h1:1pdIZTAHUz+HDKDVZ++5xg/duPlhKAIzw9qy42CWYp4= +github.com/spf13/viper v1.3.2 h1:VUFqw5KcqRf7i70GOzW7N+Q7+gxVBkSSqiXB12+JQ4M= +github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.0/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.1/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/tcnksm/go-input v0.0.0-20180404061846-548a7d7a8ee8/go.mod h1:IlWNj9v/13q7xFbaK4mbyzMNwrZLaWSHx/aibKIZuIg= -github.com/testcontainers/testcontainer-go v0.0.0-20181115231424-8e868ca12c0f/go.mod h1:SrG3IY071gtmZJjGbKO+POJ57a/MMESerYNWt6ZRtKs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/tinylib/msgp v0.0.0-20190103190839-ade0ca4ace05/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.0.2 h1:DfdQrzQa7Yh2es9SuLkixqxuXS2SxsdYn0KbdrOGWD8= github.com/tinylib/msgp v1.0.2/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= -github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= -github.com/tylerb/graceful v1.2.15/go.mod h1:LPYTbOYmUTdabwRt0TGhLllQ0MUNbs0Y5q1WXJOI9II= -github.com/willf/bitset v1.1.9/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= -github.com/xanzy/ssh-agent v0.2.0/go.mod h1:0NyE30eGUDliuLEHJgYte/zncp2zdTStcOnWhgSqHD8= -github.com/xlab/treeprint v0.0.0-20181112141820-a009c3971eca h1:1CFlNzQhALwjS9mBAUkycX616GzgsuYUOCHA5+HSlXI= -github.com/xlab/treeprint v0.0.0-20181112141820-a009c3971eca/go.mod h1:ce1O1j6UtZfjr22oyGxGLbauSBp2YVXpARAosm7dHBg= -github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= -github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= -github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= -go.opencensus.io v0.19.0 h1:+jrnNy8MR4GZXvwF9PEuSyHxA4NaTf6601oNRwCSXq0= -go.opencensus.io v0.19.0/go.mod h1:AYeH0+ZxYyghG8diqaaIq/9P3VgCCt5GF2ldCY4dkFg= +github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/twpayne/go-geom v1.0.5/go.mod h1:gO3i8BeAvZuihwwXcw8dIOWXebCzTmy3uvXj9dZG2RA= +github.com/twpayne/go-kml v1.0.0/go.mod h1:LlvLIQSfMqYk2O7Nx8vYAbSLv4K9rjMvLlEdUKWdjq0= +github.com/twpayne/go-polyline v1.0.0/go.mod h1:ICh24bcLYBX8CknfvNPKqoTbe+eg+MX1NPyJmSBo7pU= +github.com/ugorji/go v1.1.2/go.mod h1:hnLbHMwcvSihnDhEfx2/BzKp2xb0Y+ErdfYcrs9tkJQ= +github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= +github.com/ugorji/go/codec v0.0.0-20190204201341-e444a5086c43/go.mod h1:iT03XoTwV7xq/+UGwKO3UbC1nNNlopQiY61beSdrtOA= +github.com/urfave/cli v1.20.0 h1:fDqGv3UG/4jbVl/QkFwEdddtEDjh/5Ov6X+0B/3bPaw= +github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= +github.com/urfave/cli/v2 v2.2.0 h1:JTTnM6wKzdA0Jqodd966MVj4vWbbquZykeX1sKbe2C4= +github.com/urfave/cli/v2 v2.2.0/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= +github.com/vektah/dataloaden v0.2.1-0.20190515034641-a19b9a6e7c9e/go.mod h1:/HUdMve7rvxZma+2ZELQeNh88+003LL7Pf/CZ089j8U= +github.com/vektah/gqlparser v1.1.2/go.mod h1:1ycwN7Ij5njmMkPPAOaRFY4rET2Enx7IkVv3vaXspKw= +github.com/willf/bitset v0.0.0-20181014161241-71fa2377963f/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= +github.com/willf/bitset v1.1.3/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= +github.com/willf/bitset v1.1.11 h1:N7Z7E9UvjW+sGsEl7k/SJrvY2reP1A07MrGuCjIOjRE= +github.com/willf/bitset v1.1.11/go.mod h1:83CECat5yLh5zVOf4P1ErAgKA5UDvKtgyUABdr3+MjI= +github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +github.com/xlab/treeprint v0.0.0-20180616005107-d6fb6747feb6/go.mod h1:ce1O1j6UtZfjr22oyGxGLbauSBp2YVXpARAosm7dHBg= +github.com/xlab/treeprint v1.0.0 h1:J0TkWtiuYgtdlrkkrDLISYBQ92M+X5m4LrIIMKrbDTs= +github.com/xlab/treeprint v1.0.0/go.mod h1:IoImgRak9i3zJyuxOKUP1v4UZd1tMoKkq/Cimt1uhCg= +github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= +go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= +go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= +go.etcd.io/etcd v0.0.0-20190228193606-a943ad0ee4c9/go.mod h1:KSGwdbiFchh5KIC9My2+ZVl5/3ANcwohw50dpPwa2cw= +go.etcd.io/etcd v0.0.0-20200824191128-ae9734ed278b h1:QS2G6o7lP5jDfqsEdRAJM3J/5Ml5fpWbh9EUNpzKAVY= +go.etcd.io/etcd v0.0.0-20200824191128-ae9734ed278b/go.mod h1:yVHk9ub3CSBatqGNg7GRmsnfLWtoW60w4eDYfh7vHDg= +go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2 h1:75k/FF0Q2YM8QYo07VPddOLBslDt1MZOdEslOHvmzAs= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= +go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= +go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/multierr v1.1.0 h1:HoEmRHQPVSqub6w2z2d2EOVs2fjyFRGyofhKuyDq0QI= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= +go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= +go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= +go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.9.1 h1:XCJQEf3W6eZaVwhRBof6ImoYGJSITeKWsyeh3HFu/5o= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -golang.org/x/crypto v0.0.0-20180505025534-4ec37c66abab/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= +go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM= +go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= +golang.org/x/crypto v0.0.0-20180608092829-8ac0e0d97ce4/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9 h1:mKdxBk7AujPs8kU4m80U72y/zjbZ3UcXC7dClwKbUI0= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529 h1:iMGN4xG0cnqj3t+zOM8wUB0BiPKHEwSxEZCvzcbZuvk= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550 h1:ObdrDkeb4kJdCP557AjRjq69pTHfNouLtWZG7j9rPN8= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0 h1:hb9wdF1z5waM+dSIICn1l0DkLVDT3hqhhQsDNUmHPRE= +golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20181112044915-a3060d491354/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299 h1:zQpM52jfKHG6II1ISZY1ZcpygvuSFZpLwfluuF89XOg= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20181217174547-8f45f776aaf1/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f h1:J5lckAjkw6qYlOZNj90mLYNTEKDvWeuc1yieZ8qUzUE= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181217023233-e147a9138326 h1:iCzOf0xz39Tstp+Tu/WwyGjUXCk34QhQORRxBeXXTA4= -golang.org/x/net v0.0.0-20181217023233-e147a9138326/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190125091013-d26f9f9a57f3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859 h1:R/3boaszxrf1GEUWTVDzSKVwLmSJpwZ1yqXm8j0v2QI= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553 h1:efeOvDhwQ29Dj3SdAV/MJf8oukgn+8D8WgaCaRMchF8= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20201010224723-4f7140c49acb h1:mUVeFHoDKis5nxCAzoAi7E8Ghb86EXh/RK6wtvJIqRY= +golang.org/x/net v0.0.0-20201010224723-4f7140c49acb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d h1:TzXSXBo42m9gQenoE3b9BGiEpg5IG2JkU5FkPIawgtw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4 h1:YUO/7uOKsKeq9UokNS62b8FYywz3ker1l1vDZRCRefw= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180903190138-2b024373dcd9/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180906133057-8cf3aee42992/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181030150119-7e31e0c00fa0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181218192612-074acd46bca6 h1:MXtOG7w2ND9qNCUZSDBGll/SpVIq7ftozR9I8/JGBHY= -golang.org/x/sys v0.0.0-20181218192612-074acd46bca6/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190405154228-4b34438f7a67/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d h1:+R4KGOnez64A81RvjARKc4UT5/tI9ujCIVX+P5KiHuI= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191210023423-ac6580df4449/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200107162124-548cf772de50 h1:YvQ10rzcqWXLlJZ3XCUoO25savxmscf4+SC+ZqiCHhA= +golang.org/x/sys v0.0.0-20200107162124-548cf772de50/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f h1:+Nyd8tzPX9R7BWHguqsrbFdRx3WQ/1ib8I44HXV5yTA= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c h1:fqgJT0MGcGpPgpWU7VRdRjuArfcOvC4AoJmILihzhDg= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s= +golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181219222714-6e267b5cc78e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181221154417-3ad2d988d5e2/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190125232054-d66bd3c5d5a6/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190515012406-7d7faa4812bd/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5 h1:hKsoRgsbwY1NafxrwTs+k64bikrLBkAgPir1TNCj3Zs= +golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200108203644-89082a384178 h1:f5gMxb6FbpY48csegk9UPd7IAHVrBD013CU7N4pWzoE= +golang.org/x/tools v0.0.0-20200108203644-89082a384178/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= +gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca h1:PupagGYwj8+I4ubCxcmcBRk3VlUWtTg5huQpZR9flmE= gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= +gonum.org/v1/gonum v0.6.0 h1:DJy6UzXbahnGUf1ujUNkh/NEtK14qMo2nvlBPs4U5yw= +gonum.org/v1/gonum v0.6.0/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= +gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6 h1:4WsZyVtkthqrHTbDCJfiTs8IWNYE4uvsSDgaV6xpp+o= gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= -google.golang.org/api v0.0.0-20181021000519-a2651947f503/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= -google.golang.org/api v0.0.0-20181220000619-583d854617af/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= +gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0 h1:OE9mWmgKkjJyEmDAAtGMPjXu+YNeGvK9VTSHY6+Qihc= +gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= +gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= +google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= +google.golang.org/api v0.3.2/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0 h1:yzlyyDW/J0w8yNFJIhiAJy4kq74S+1DOLdawELNxFMA= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5 h1:tycE03LOZYQNhDpS27tcQdAzLCVMaj7QT2SXxebnpCM= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180608181217-32ee49c4dd80/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181016170114-94acd270e44e/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181219182458-5a97ab628bfb h1:dQshZyyJ5W/Xk8myF4GKBak1pZW6EywJuQ8+44EQhGA= -google.golang.org/genproto v0.0.0-20181219182458-5a97ab628bfb/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190516172635-bb713bdc0e52/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= +google.golang.org/genproto v0.0.0-20190716160619-c506a9f90610/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200108215221-bd8f9a0ef82f h1:2wh8dWY8959cBGQvk1RD+/eQBgRYYDaZ+hT0/zsARoA= +google.golang.org/genproto v0.0.0-20200108215221-bd8f9a0ef82f/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= -google.golang.org/grpc v1.15.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= -google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= -google.golang.org/grpc v1.17.0 h1:TRJYBgMclJvGYn2rIMjj+h9KtMt5r1Ij7ODVRIZkwhk= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= -gopkg.in/asn1-ber.v1 v1.0.0-20170511165959-379148ca0225/go.mod h1:cuepJuh7vyXfUyUwEgHQXw849cJrilpS5NeIjOWESAw= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +gopkg.in/DataDog/dd-trace-go.v1 v1.13.1 h1:oTzOClfuudNhW9Skkp2jxjqYO92uDKXqKLbiuPA13Rk= +gopkg.in/DataDog/dd-trace-go.v1 v1.13.1/go.mod h1:DVp8HmDh8PuTu2Z0fVVlBsyWaC++fzwVCaGWylTe3tg= +gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= +gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/fatih/pool.v2 v2.0.0 h1:xIFeWtxifuQJGk/IEPKsTduEKcKvPmhoiVDGpC40nKg= -gopkg.in/fatih/pool.v2 v2.0.0/go.mod h1:8xVGeu1/2jr2wm5V9SPuMht2H5AEmf5aFMGSQixtjTY= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= -gopkg.in/ldap.v2 v2.5.1/go.mod h1:oI0cpe/D7HRtBQl8aTg+ZmzFUAvu4lsv3eLXMLGFxWk= -gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= -gopkg.in/robfig/cron.v2 v2.0.0-20150107220207-be2e0b0deed5/go.mod h1:hiOFpYm0ZJbusNj2ywpbrXowU3G8U6GIQzqn2mw1UIE= -gopkg.in/src-d/go-billy.v4 v4.2.1/go.mod h1:tm33zBoOwxjYHZIE+OV8bxTWFMJLrconzFMd38aARFk= -gopkg.in/src-d/go-git-fixtures.v3 v3.1.1/go.mod h1:dLBcvytrw/TYZsNTWCnkNF2DSIlzWYqTe3rJR56Ac7g= -gopkg.in/src-d/go-git.v4 v4.8.1/go.mod h1:Vtut8izDyrM8BUVQnzJ+YvmNcem2J89EmfZYCkLokZk= +gopkg.in/gemnasium/logrus-airbrake-hook.v2 v2.1.2/go.mod h1:Xk6kEKp8OKb+X14hQBKWaSkCsqBpgog8nAV2xsGOxlo= +gopkg.in/ini.v1 v1.48.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= +gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= +gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= -gopkg.in/vmihailenco/msgpack.v2 v2.9.1/go.mod h1:/3Dn1Npt9+MYyLpYYXjInO/5jvMLamn+AEGwNEOatn8= -gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= +gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= +gopkg.in/yaml.v2 v2.2.4 h1:/eiJrUcujPVeJ3xlSWaiNi3uSVmDGBK1pDHUHAnao1I= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20180920025451-e3ad64cb4ed3/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20181108184350-ae8f1f9103cc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -labix.org/v2/mgo v0.0.0-20140701140051-000000000287/go.mod h1:Lg7AYkt1uXJoR9oeSZ3W/8IXLdvOfIITgZnommstyz4= -launchpad.net/gocheck v0.0.0-20140225173054-000000000087/go.mod h1:hj7XX3B/0A+80Vse0e+BUHsHMTEhd0O4cpUHr/e/BUM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +rsc.io/binaryregexp v0.2.0 h1:HfqmD5MEmC0zvwBuF187nq9mdnXjXsSivRiXN7SmRkE= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= +sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= +sourcegraph.com/sourcegraph/appdash v0.0.0-20180110180208-2cc67fd64755/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= +sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:L5q+DGLGOQFpo1snNEkLOJT2d1YTW66rWNzatr3He1k= diff --git a/logging/config.go b/logging/config.go new file mode 100644 index 0000000..7f3f228 --- /dev/null +++ b/logging/config.go @@ -0,0 +1,8 @@ +package logging + +type Config struct { + Format string + Level string + Dir string + FileName string +} diff --git a/logging/log.go b/logging/log.go new file mode 100644 index 0000000..d7a140d --- /dev/null +++ b/logging/log.go @@ -0,0 +1,92 @@ +package logging + +import ( + "fmt" + "io" + "os" + "path/filepath" + "strings" + "time" + + "github.com/influxdata/influxdb/pkg/snowflake" + zaplogfmt "github.com/jsternberg/zap-logfmt" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "gopkg.in/natefinch/lumberjack.v2" +) + +var ( + gen = snowflake.New(0) +) + +const TimeFormat = "2006-01-02 15:04:05" + +func newEncoderConfig() zapcore.EncoderConfig { + config := zap.NewProductionEncoderConfig() + config.EncodeTime = func(ts time.Time, encoder zapcore.PrimitiveArrayEncoder) { + encoder.AppendString(ts.Local().Format(TimeFormat)) + } + config.EncodeDuration = func(d time.Duration, encoder zapcore.PrimitiveArrayEncoder) { + val := float64(d) / float64(time.Millisecond) + encoder.AppendString(fmt.Sprintf("%.3fms", val)) + } + config.LevelKey = "lvl" + return config +} + +func nextID() string { + return gen.NextString() +} + +func newEncoder(format string) (zapcore.Encoder, error) { + config := newEncoderConfig() + switch format { + case "json": + return zapcore.NewJSONEncoder(config), nil + case "console", "auto": + return zapcore.NewConsoleEncoder(config), nil + case "logfmt": + return zaplogfmt.NewEncoder(config), nil + default: + return nil, fmt.Errorf("unknown logging format: %s", format) + } +} + +func createWriter(c *Config) io.Writer { + if c.Dir != "" { + dir := strings.TrimRight(c.Dir, string(filepath.Separator)) + return &lumberjack.Logger{ + Filename: filepath.Join(dir, c.FileName), + MaxSize: 100, + MaxBackups: 5, + Compress: true, + } + } else { + return os.Stderr + } +} + +func InitialLogging(c *Config) (*zap.Logger, error) { + encoder, err := newEncoder(c.Format) + if err != nil { + return nil, err + } + lvl := zap.InfoLevel + switch strings.ToLower(c.Level) { + case "info": + lvl = zap.InfoLevel + case "warn", "warning": + lvl = zap.WarnLevel + case "debug": + lvl = zap.DebugLevel + case "fatal": + lvl = zap.FatalLevel + case "panic": + lvl = zap.PanicLevel + } + return zap.New(zapcore.NewCore( + encoder, + zapcore.Lock(zapcore.AddSync(createWriter(c))), + lvl, + ), zap.Fields(zap.String("log_id", nextID()))), nil +} diff --git a/raftmeta/apply.go b/raftmeta/apply.go index e7bb881..773d0ad 100644 --- a/raftmeta/apply.go +++ b/raftmeta/apply.go @@ -4,11 +4,13 @@ import ( "encoding/json" "errors" "fmt" + "time" + "github.com/angopher/chronus/raftmeta/internal" + imeta "github.com/angopher/chronus/services/meta" "github.com/angopher/chronus/x" "github.com/influxdata/influxdb/services/meta" "go.uber.org/zap" - "time" ) func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) error { @@ -17,7 +19,7 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err s.ApplyCallBack(proposal, index) } msgName, _ := internal.MessageTypeName[proposal.Type] - s.Logger.Info("applyCommitted ", zap.String("type", msgName)) + s.Logger.Debug("applyCommitted ", zap.String("type", msgName)) pctx := s.props.pctx(proposal.Key) if pctx == nil { @@ -30,8 +32,8 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req CreateDatabaseReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("apply create database %+v", req)) - db, err := s.MetaCli.CreateDatabase(req.Name) + s.SugaredLogger.Infof("apply create database %+v", req) + db, err := s.MetaStore.CreateDatabase(req.Name) pctx.err = err if err == nil && pctx.retData != nil { x.AssertTrue(db != nil) @@ -42,20 +44,20 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req DropDatabaseReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DropDatabase(req.Name) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.DropDatabase(req.Name) case internal.DropRetentionPolicy: var req DropRetentionPolicyReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DropRetentionPolicy(req.Database, req.Policy) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.DropRetentionPolicy(req.Database, req.Policy) case internal.CreateShardGroup: var req CreateShardGroupReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - sg, err := s.MetaCli.CreateShardGroup(req.Database, req.Policy, time.Unix(req.Timestamp, 0)) + s.SugaredLogger.Debugf("req %+v", req) + sg, err := s.MetaStore.CreateShardGroup(req.Database, req.Policy, time.Unix(req.Timestamp, 0)) if err == nil && pctx.retData != nil { if sg != nil { *pctx.retData.(*meta.ShardGroupInfo) = *sg @@ -68,8 +70,8 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req CreateDataNodeReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - ni, err := s.MetaCli.CreateDataNode(req.HttpAddr, req.TcpAddr) + s.SugaredLogger.Debugf("req %+v", req) + ni, err := s.MetaStore.CreateDataNode(req.HttpAddr, req.TcpAddr) if err == nil && pctx.retData != nil { x.AssertTrue(ni != nil) *pctx.retData.(*meta.NodeInfo) = *ni @@ -79,13 +81,13 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req DeleteDataNodeReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DeleteDataNode(req.Id) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.DeleteDataNode(req.Id) case internal.CreateRetentionPolicy: var req CreateRetentionPolicyReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) + s.SugaredLogger.Debugf("req %+v", req) var duration *time.Duration if req.Rps.Duration > 0 { @@ -98,7 +100,7 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err Duration: duration, ShardGroupDuration: req.Rps.ShardGroupDuration, } - rpi, err := s.MetaCli.CreateRetentionPolicy(req.Database, &spec, req.MakeDefault) + rpi, err := s.MetaStore.CreateRetentionPolicy(req.Database, &spec, req.MakeDefault) if err == nil && pctx.retData != nil { x.AssertTrue(rpi != nil) *pctx.retData.(*meta.RetentionPolicyInfo) = *rpi @@ -109,7 +111,7 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req UpdateRetentionPolicyReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) + s.SugaredLogger.Debugf("req %+v", req) var duration *time.Duration if req.Rps.Duration > 0 { @@ -132,13 +134,13 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err Duration: duration, ShardGroupDuration: sduration, } - return s.MetaCli.UpdateRetentionPolicy(req.Database, req.Name, &up, req.MakeDefault) + return s.MetaStore.UpdateRetentionPolicy(req.Database, req.Name, &up, req.MakeDefault) case internal.CreateDatabaseWithRetentionPolicy: var req CreateDatabaseWithRetentionPolicyReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) + s.SugaredLogger.Debugf("req %+v", req) var duration *time.Duration if req.Rps.Duration > 0 { @@ -151,7 +153,7 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err Duration: duration, ShardGroupDuration: req.Rps.ShardGroupDuration, } - db, err := s.MetaCli.CreateDatabaseWithRetentionPolicy(req.Name, &spec) + db, err := s.MetaStore.CreateDatabaseWithRetentionPolicy(req.Name, &spec) if err == nil && pctx.retData != nil { x.AssertTrue(db != nil) *pctx.retData.(*meta.DatabaseInfo) = *db @@ -162,8 +164,8 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req CreateUserReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - user, err := s.MetaCli.CreateUser(req.Name, req.Password, req.Admin) + s.SugaredLogger.Debugf("req %+v", req) + user, err := s.MetaStore.CreateUser(req.Name, req.Password, req.Admin) if err == nil && pctx.retData != nil { x.AssertTrue(user != nil) *pctx.retData.(*meta.UserInfo) = *(user.(*meta.UserInfo)) @@ -174,37 +176,38 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req DropUserReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DropUser(req.Name) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.DropUser(req.Name) case internal.UpdateUser: var req UpdateUserReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.UpdateUser(req.Name, req.Password) + s.SugaredLogger.Debugf("req %+v", req) + // XXX password should be hashed before to keep consistent between nodes + return s.MetaStore.UpdateUser(req.Name, req.Password) case internal.SetPrivilege: var req SetPrivilegeReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.SetPrivilege(req.UserName, req.Database, req.Privilege) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.SetPrivilege(req.UserName, req.Database, req.Privilege) case internal.SetAdminPrivilege: var req SetAdminPrivilegeReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.SetAdminPrivilege(req.UserName, req.Admin) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.SetAdminPrivilege(req.UserName, req.Admin) case internal.Authenticate: var req AuthenticateReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - user, err := s.MetaCli.Authenticate(req.UserName, req.Password) - if err == nil { + s.SugaredLogger.Debugf("req %+v", req) + user, err := s.MetaStore.Authenticate(req.UserName, req.Password) + if err == nil && pctx != nil && pctx.retData != nil { x.AssertTrue(user != nil) *pctx.retData.(*meta.UserInfo) = *(user.(*meta.UserInfo)) } @@ -214,81 +217,98 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err var req AddShardOwnerReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("add shard owner req %+v", req)) - return s.MetaCli.AddShardOwner(req.ShardID, req.NodeID) + s.SugaredLogger.Debugf("add shard owner req %+v", req) + return s.MetaStore.AddShardOwner(req.ShardID, req.NodeID) case internal.RemoveShardOwner: var req RemoveShardOwnerReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("remove shard owner req %+v", req)) - return s.MetaCli.RemoveShardOwner(req.ShardID, req.NodeID) + s.SugaredLogger.Debugf("remove shard owner req %+v", req) + return s.MetaStore.RemoveShardOwner(req.ShardID, req.NodeID) case internal.DropShard: var req DropShardReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DropShard(req.Id) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.DropShard(req.Id) case internal.TruncateShardGroups: var req TruncateShardGroupsReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.TruncateShardGroups(req.Time) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.TruncateShardGroups(req.Time) case internal.PruneShardGroups: - return s.MetaCli.PruneShardGroups() + var req PruneShardGroupsReq + if len(proposal.Data) > 1 { + err := json.Unmarshal(proposal.Data, &req) + x.Check(err) + s.SugaredLogger.Debugf("req %+v", req) + } + if req.Expiration.IsZero() { + // fallback + req.Expiration = time.Now().Add(imeta.SHARDGROUP_INFO_EVICTION) + } + return s.MetaStore.PruneShardGroups(req.Expiration) case internal.DeleteShardGroup: var req DeleteShardGroupReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DeleteShardGroup(req.Database, req.Policy, req.Id) + if req.Now.IsZero() { + req.Now = time.Now() + } + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.DeleteShardGroup(req.Database, req.Policy, req.Id, req.Now) case internal.PrecreateShardGroups: var req PrecreateShardGroupsReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.PrecreateShardGroups(req.From, req.To) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.PrecreateShardGroups(req.From, req.To) case internal.CreateContinuousQuery: var req CreateContinuousQueryReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.CreateContinuousQuery(req.Database, req.Name, req.Query) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.CreateContinuousQuery(req.Database, req.Name, req.Query) case internal.DropContinuousQuery: var req DropContinuousQueryReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DropContinuousQuery(req.Database, req.Name) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.DropContinuousQuery(req.Database, req.Name) case internal.CreateSubscription: var req CreateSubscriptionReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.CreateSubscription(req.Database, req.Rp, req.Name, req.Mode, req.Destinations) + s.SugaredLogger.Debugf("req %+v", req) + return s.MetaStore.CreateSubscription(req.Database, req.Rp, req.Name, req.Mode, req.Destinations) case internal.DropSubscription: var req DropSubscriptionReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - return s.MetaCli.DropSubscription(req.Database, req.Rp, req.Name) + s.SugaredLogger.Debug("req %+v", req) + return s.MetaStore.DropSubscription(req.Database, req.Rp, req.Name) case internal.AcquireLease: var req AcquireLeaseReq err := json.Unmarshal(proposal.Data, &req) x.Check(err) - s.Logger.Info(fmt.Sprintf("req %+v", req)) - lease, err := s.leases.Acquire(req.Name, req.NodeId) + s.SugaredLogger.Debugf("req %+v", req) + if req.RequestTime == 0 { + // XXX compatible with old data format + req.RequestTime = time.Now().UnixNano() / 1e6 + } + lease, err := s.leases.Acquire(req.Name, req.NodeId, req.RequestTime) if err == nil && pctx.retData != nil { x.AssertTrue(lease != nil) *pctx.retData.(*meta.Lease) = *lease @@ -296,7 +316,7 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err return err case internal.SnapShot: - md, err := s.MetaCli.MarshalBinary() + md, err := s.MetaStore.MarshalBinary() x.Check(err) var sndata internal.SnapshotData sndata.Data = md @@ -311,14 +331,14 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err case internal.CreateChecksumMsg: //TODO:optimize, reduce block time start := time.Now() - mcd := s.MetaCli.Data() + mcd := s.MetaStore.Data() //消除DeleteAt和TruncatedAt对checksum的影响 - for i, _ := range mcd.Databases { + for i := range mcd.Databases { db := &mcd.Databases[i] - for j, _ := range db.RetentionPolicies { + for j := range db.RetentionPolicies { rp := &db.RetentionPolicies[j] - for k, _ := range rp.ShardGroups { + for k := range rp.ShardGroups { sg := &rp.ShardGroups[k] sg.DeletedAt = time.Unix(0, 0) sg.TruncatedAt = time.Unix(0, 0) @@ -331,35 +351,54 @@ func (s *RaftNode) applyCommitted(proposal *internal.Proposal, index uint64) err s.lastChecksum.checksum = x.Md5(data) s.lastChecksum.needVerify = true - detail := fmt.Sprintf("index:%d, checksum:%s, data:%+v", index, s.lastChecksum.checksum, mcd) - s.Logger.Info(fmt.Sprintf("create checksum costs:%s detail:%s", time.Now().Sub(start), detail)) + s.Logger.Debug( + fmt.Sprintf("create checksum costs:%s detail:%s", + time.Now().Sub(start), + fmt.Sprintf("index:%d, checksum:%s, data:%+v", + index, s.lastChecksum.checksum, mcd, + ), + ), + ) case internal.VerifyChecksumMsg: start := time.Now() var req internal.VerifyChecksum err := json.Unmarshal(proposal.Data, &req) x.Check(err) if req.NodeID == s.ID { - s.Logger.Info("ignore checksum. self trigger this verify") + s.Logger.Debug("ignore checksum. self trigger this verify") s.lastChecksum.needVerify = false return nil } if s.lastChecksum.index == 0 { //have no checksum only when restart - s.Logger.Info("ignore checksum. have no checksum", zap.Uint64("index", req.Index)) + s.Logger.Warn("ignore checksum. have no checksum", zap.Uint64("index", req.Index)) return nil } if s.lastChecksum.index != req.Index { - s.Logger.Warn("ingore checksum", zap.Uint64("last index:", s.lastChecksum.index), zap.Uint64("index", req.Index)) + s.Logger.Warn("ignore checksum", zap.Uint64("last index:", s.lastChecksum.index), zap.Uint64("index", req.Index)) return nil } s.Logger.Info("checksum", zap.Uint64("index", req.Index), zap.String("checksum", s.lastChecksum.checksum)) - x.AssertTruef(s.lastChecksum.checksum == req.Checksum, "verify checksum fail") + x.AssertTruef(s.lastChecksum.checksum == req.Checksum, "verify checksum fail, local %s != %s, data=%+v", + s.lastChecksum.checksum, req.Checksum, + s.MetaStore.Data(), + ) s.Logger.Info(fmt.Sprintf("verify checksum success. costs %s", time.Now().Sub(start))) + case internal.FreezeDataNode: + var req FreezeDataNodeReq + err := json.Unmarshal(proposal.Data, &req) + x.Check(err) + s.SugaredLogger.Debugf("req %+v", req) + if req.Freeze { + return s.MetaStore.FreezeDataNode(req.Id) + } else { + return s.MetaStore.UnfreezeDataNode(req.Id) + } default: - return fmt.Errorf("Unkown msg type:%d", proposal.Type) + return fmt.Errorf("Unknown msg type:%d", proposal.Type) } return nil diff --git a/raftmeta/badger_logger_bridge.go b/raftmeta/badger_logger_bridge.go new file mode 100644 index 0000000..fc2e0fb --- /dev/null +++ b/raftmeta/badger_logger_bridge.go @@ -0,0 +1,35 @@ +package raftmeta + +import ( + "strings" + + "github.com/dgraph-io/badger/v2" + "go.uber.org/zap" +) + +type badgerLoggerBridge struct { + badger.Logger + suger *zap.SugaredLogger +} + +func NewBadgerLoggerBridge(logger *zap.Logger) *badgerLoggerBridge { + return &badgerLoggerBridge{ + suger: logger.Sugar(), + } +} + +func (b *badgerLoggerBridge) Errorf(format string, args ...interface{}) { + b.suger.Errorf(strings.Trim(format, "\n"), args...) +} + +func (b *badgerLoggerBridge) Infof(format string, args ...interface{}) { + b.suger.Infof(strings.Trim(format, "\n"), args...) +} + +func (b *badgerLoggerBridge) Warningf(format string, args ...interface{}) { + b.suger.Warnf(strings.Trim(format, "\n"), args...) +} + +func (b *badgerLoggerBridge) Debugf(format string, args ...interface{}) { + b.suger.Debugf(strings.Trim(format, "\n"), args...) +} diff --git a/raftmeta/cluster_leases.go b/raftmeta/cluster_leases.go new file mode 100644 index 0000000..d579c63 --- /dev/null +++ b/raftmeta/cluster_leases.go @@ -0,0 +1,95 @@ +package raftmeta + +import ( + "errors" + "sync" + "time" + + "github.com/influxdata/influxdb/services/meta" + "go.uber.org/zap" +) + +var ( + ErrSkipExpired = errors.New("Skip an expired lease") + ErrAcquiredByOther = errors.New("Another node has the lease") +) + +// ClusterLeases is a concurrency-safe collection of leases keyed by name. +type ClusterLeases struct { + mu sync.Mutex + m map[string]*meta.Lease + d time.Duration + sugar *zap.SugaredLogger +} + +// NewLeases returns a new instance of Leases. +func NewClusterLeases(d time.Duration, logger *zap.Logger) *ClusterLeases { + return &ClusterLeases{ + m: make(map[string]*meta.Lease), + d: d, + sugar: logger.Sugar(), + } +} + +func (leases *ClusterLeases) Get(name string) *meta.Lease { + leases.mu.Lock() + defer leases.mu.Unlock() + + return leases.m[name] +} + +// CanAcquire returns whether it's necessary actually trying to acquire for now. +func (leases *ClusterLeases) ShouldTry(name string, nodeID uint64) bool { + leases.mu.Lock() + defer leases.mu.Unlock() + + now := time.Now() + l := leases.m[name] + if l != nil { + if now.After(l.Expiration) || l.Owner == nodeID { + return true + } + return false + } + + return true +} + +// Acquire acquires a lease with the given name for the given nodeID. +// If the lease doesn't exist or exists but is expired, a valid lease is returned. +// If nodeID already owns the named and unexpired lease, the lease expiration is extended. +// If a different node owns the lease, an error is returned. +func (leases *ClusterLeases) Acquire(name string, nodeID uint64, requestTime int64) (*meta.Lease, error) { + leases.mu.Lock() + defer leases.mu.Unlock() + + now := time.Now() + beginTime := time.Unix(requestTime/1000, (requestTime%1000)*1e6) + expireTime := beginTime.Add(leases.d) + + if now.After(expireTime) { + // skip + leases.sugar.Warnf("Skip expired lock: key=%s, node=%d", name, nodeID) + return nil, ErrSkipExpired + } + + l := leases.m[name] + if l != nil { + if now.After(l.Expiration) || l.Owner == nodeID { + l.Expiration = expireTime + l.Owner = nodeID + return l, nil + } + return l, ErrAcquiredByOther + } + + l = &meta.Lease{ + Name: name, + Expiration: expireTime, + Owner: nodeID, + } + + leases.m[name] = l + + return l, nil +} diff --git a/raftmeta/cluster_leases_test.go b/raftmeta/cluster_leases_test.go new file mode 100644 index 0000000..1805e3a --- /dev/null +++ b/raftmeta/cluster_leases_test.go @@ -0,0 +1,37 @@ +package raftmeta_test + +import ( + "testing" + "time" + + "github.com/angopher/chronus/raftmeta" + "github.com/stretchr/testify/assert" + "go.uber.org/zap" +) + +func TestLease(t *testing.T) { + leases := raftmeta.NewClusterLeases(time.Second, zap.NewNop()) + KEY1 := "k1" + NODE1 := uint64(1) + NODE2 := uint64(2) + + l, err := leases.Acquire(KEY1, NODE1, 0) + assert.Equal(t, raftmeta.ErrSkipExpired, err) + assert.Nil(t, l) + + l, err = leases.Acquire(KEY1, NODE1, time.Now().UnixNano()/1e6) + assert.Nil(t, err) + assert.NotNil(t, l) + + l, err = leases.Acquire(KEY1, NODE2, time.Now().UnixNano()/1e6) + assert.Equal(t, raftmeta.ErrAcquiredByOther, err) + assert.NotNil(t, l) + + assert.True(t, leases.ShouldTry(KEY1, NODE1)) + assert.False(t, leases.ShouldTry(KEY1, NODE2)) + + l = leases.Get(KEY1) + assert.NotNil(t, l) + assert.Equal(t, KEY1, l.Name) + assert.Equal(t, NODE1, l.Owner) +} diff --git a/raftmeta/config.go b/raftmeta/config.go index f6e39df..64565f1 100644 --- a/raftmeta/config.go +++ b/raftmeta/config.go @@ -1,11 +1,12 @@ package raftmeta import ( + "io/ioutil" + "net" + "github.com/BurntSushi/toml" "golang.org/x/text/encoding/unicode" "golang.org/x/text/transform" - "io/ioutil" - "net" ) const ( @@ -37,6 +38,10 @@ type Config struct { SnapshotIntervalSec int `toml:"snapshot-interval"` ChecksumIntervalSec int `toml:"checksum-interval"` RetentionAutoCreate bool `toml:"retention-auto-create"` + + LogFormat string `toml:"log-format"` + LogLevel string `toml:"log-level"` + LogDir string `toml:"log-dir"` } type Peer struct { @@ -52,15 +57,18 @@ func NewConfig() Config { MyAddr: DefaultAddr, RaftId: 1, Peers: []Peer{}, - TickTimeMs: 20, + TickTimeMs: 60, ElectionTick: DefaultElectionTick, HeartbeatTick: DefaultHeartbeatTick, MaxSizePerMsg: DefaultMaxSizePerMsg, MaxInflightMsgs: DefaultMaxInflightMsgs, WalDir: "./wal", - SnapshotIntervalSec: 60, - ChecksumIntervalSec: 10, + SnapshotIntervalSec: 300, + ChecksumIntervalSec: 120, RetentionAutoCreate: true, + LogFormat: "console", + LogLevel: "info", + LogDir: "./logs", } } diff --git a/raftmeta/internal/message.go b/raftmeta/internal/message.go index e75bf3f..1a55467 100644 --- a/raftmeta/internal/message.go +++ b/raftmeta/internal/message.go @@ -1,7 +1,7 @@ package internal import ( - "github.com/coreos/etcd/raft/raftpb" + "go.etcd.io/etcd/raft/raftpb" ) const ( @@ -36,6 +36,7 @@ const ( VerifyChecksumMsg = 29 AddShardOwner = 30 RemoveShardOwner = 31 + FreezeDataNode = 32 ) var MessageTypeName = map[int]string{ @@ -70,6 +71,7 @@ var MessageTypeName = map[int]string{ 29: "VerifyChecksumMsg", 30: "AddShardOwner", 31: "RemoveShardOwner", + 32: "FreezeDataNode", } type Proposal struct { diff --git a/raftmeta/linearizable_read.go b/raftmeta/linearizable_read.go index 9bddc39..12bceaf 100644 --- a/raftmeta/linearizable_read.go +++ b/raftmeta/linearizable_read.go @@ -6,9 +6,10 @@ import ( "encoding/binary" "errors" "fmt" - "github.com/coreos/etcd/raft" "sync" "time" + + "go.etcd.io/etcd/raft" ) type Linearizabler struct { diff --git a/raftmeta/meta_service.go b/raftmeta/meta_service.go index 3d901a6..ccde66e 100644 --- a/raftmeta/meta_service.go +++ b/raftmeta/meta_service.go @@ -9,11 +9,14 @@ import ( "strings" "time" + _ "net/http/pprof" + "github.com/angopher/chronus/raftmeta/internal" imeta "github.com/angopher/chronus/services/meta" "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxql" "go.uber.org/zap" + "golang.org/x/crypto/bcrypt" ) type CommonResp struct { @@ -21,6 +24,30 @@ type CommonResp struct { RetMsg string `json:"ret_msg"` } +type NodeStatus struct { + ID uint64 `json:"id"` + Addr string `json:"addr"` + Vote uint64 `json:"vote"` + Match uint64 `json:"match"` + Next uint64 `json:"next"` + Role string `json:"role"` + Progress string `json:"progress"` +} + +type StatusNodeResp struct { + CommonResp + Status NodeStatus `json:"status"` +} + +type StatusClusterResp struct { + CommonResp + Term uint64 `json:"term"` + Commit uint64 `json:"commit"` + Applied uint64 `json:"applied"` + Leader uint64 `json:"leader"` + Nodes []NodeStatus `json:"nodes"` +} + type MetaService struct { Logger *zap.Logger Addr string @@ -55,6 +82,12 @@ func (s *MetaService) InitRouter() { http.HandleFunc("/update_cluster", func(w http.ResponseWriter, r *http.Request) { s.Node.HandleUpdateCluster(w, r) }) + http.HandleFunc("/status_cluster", func(w http.ResponseWriter, r *http.Request) { + s.Node.HandleStatusCluster(w, r) + }) + http.HandleFunc("/status_node", func(w http.ResponseWriter, r *http.Request) { + s.Node.HandleStatusNode(w, r) + }) initHttpHandler(s) } @@ -77,18 +110,7 @@ func (s *MetaService) ProposeAndWait(msgType int, data []byte, retData interface pr := &internal.Proposal{Type: msgType} pr.Data = data - resCh := make(chan error) - go func() { - err := s.Node.ProposeAndWait(ctx, pr, retData) - resCh <- err - }() - - var err error - select { - case err = <-resCh: - } - - return err + return s.Node.ProposeAndWait(ctx, pr, retData) } type CreateDatabaseReq struct { @@ -359,7 +381,7 @@ func (s *MetaService) DeleteDataNode(w http.ResponseWriter, r *http.Request) { resp.RetCode = 0 resp.RetMsg = "ok" - s.Logger.Info("DeleteDataNode ok", zap.Uint64("id", req.Id)) + s.Logger.Info(fmt.Sprintf("DeleteDataNode ok, id=%d", req.Id)) } type RetentionPolicySpec struct { @@ -566,6 +588,15 @@ func (s *MetaService) CreateUser(w http.ResponseWriter, r *http.Request) { s.Logger.Error("CreateUser fail", zap.Error(err)) return } + // regenerate data due to pre-hashed password + hash, err := bcrypt.GenerateFromPassword([]byte(req.Password), bcrypt.DefaultCost) + if err != nil { + resp.RetMsg = err.Error() + s.Logger.Error("Hash user password fail", zap.Error(err)) + return + } + req.Password = string(hash) + data, _ = json.Marshal(&req) user := &meta.UserInfo{} err = s.ProposeAndWait(internal.CreateUser, data, user) @@ -573,7 +604,6 @@ func (s *MetaService) CreateUser(w http.ResponseWriter, r *http.Request) { resp.RetMsg = err.Error() s.Logger.Error("CreateUser fail", zap.String("Name", req.Name), - zap.String("Password", req.Password), zap.Bool("Admin", req.Admin), zap.Error(err)) return @@ -584,7 +614,6 @@ func (s *MetaService) CreateUser(w http.ResponseWriter, r *http.Request) { resp.RetMsg = "ok" s.Logger.Info("CreateUser ok", zap.String("Name", req.Name), - zap.String("Password", req.Password), zap.Bool("Admin", req.Admin)) } @@ -654,12 +683,20 @@ func (s *MetaService) UpdateUser(w http.ResponseWriter, r *http.Request) { s.Logger.Error("UpdateUser fail", zap.Error(err)) return } + // regenerate data due to pre-hashed password + hash, err := bcrypt.GenerateFromPassword([]byte(req.Password), bcrypt.DefaultCost) + if err != nil { + resp.RetMsg = err.Error() + s.Logger.Error("Hash user password fail", zap.Error(err)) + return + } + req.Password = string(hash) + data, _ = json.Marshal(&req) err = s.ProposeAndWait(internal.UpdateUser, data, nil) if err != nil { s.Logger.Error("UpdateUser fail", zap.String("Name", req.Name), - zap.String("Password", req.Password), zap.Error(err)) resp.RetMsg = err.Error() return @@ -668,8 +705,7 @@ func (s *MetaService) UpdateUser(w http.ResponseWriter, r *http.Request) { resp.RetCode = 0 resp.RetMsg = "ok" s.Logger.Info("UpdateUser ok", - zap.String("Name", req.Name), - zap.String("Password", req.Password)) + zap.String("Name", req.Name)) } type SetPrivilegeReq struct { @@ -791,24 +827,18 @@ func (s *MetaService) Authenticate(w http.ResponseWriter, r *http.Request) { s.Logger.Error("Authenticate fail", zap.Error(err)) return } - - user := &meta.UserInfo{} - err = s.ProposeAndWait(internal.Authenticate, data, user) + u, err := s.Node.MetaStore.Authenticate(req.UserName, req.Password) if err != nil { s.Logger.Error("Authenticate fail", zap.String("UserName", req.UserName), - zap.String("Password", req.Password), zap.Error(err)) resp.RetMsg = err.Error() return } - resp.UserInfo = *(user) + resp.UserInfo = *(u.(*meta.UserInfo)) resp.RetCode = 0 resp.RetMsg = "ok" - s.Logger.Info("Authenticate ok", - zap.String("UserName", req.UserName), - zap.String("Password", req.Password)) } type AddShardOwnerReq struct { @@ -979,6 +1009,10 @@ func (s *MetaService) TruncateShardGroups(w http.ResponseWriter, r *http.Request s.Logger.Info("TruncateShardGroups ok", zap.Time("Time", req.Time)) } +type PruneShardGroupsReq struct { + Expiration time.Time +} + type PruneShardGroupsResp struct { CommonResp } @@ -989,7 +1023,11 @@ func (s *MetaService) PruneShardGroups(w http.ResponseWriter, r *http.Request) { resp.RetMsg = "fail" defer WriteResp(w, &resp) - err := s.ProposeAndWait(internal.PruneShardGroups, []byte{}, nil) + req := &PruneShardGroupsReq{ + Expiration: time.Now().Add(imeta.SHARDGROUP_INFO_EVICTION), + } + data, _ := json.Marshal(req) + err := s.ProposeAndWait(internal.PruneShardGroups, data, nil) if err != nil { resp.RetMsg = err.Error() s.Logger.Error("PruneShardGroups fail", zap.Error(err)) @@ -1006,6 +1044,7 @@ type DeleteShardGroupReq struct { Database string Policy string Id uint64 + Now time.Time } type DeleteShardGroupResp struct { CommonResp @@ -1020,21 +1059,22 @@ func (s *MetaService) DeleteShardGroup(w http.ResponseWriter, r *http.Request) { data, err := ioutil.ReadAll(r.Body) if err != nil { resp.RetMsg = err.Error() - s.Logger.Error("TruncateShardGroups fail", zap.Error(err)) + s.Logger.Error("DeleteShardGroup fail", zap.Error(err)) return } var req DeleteShardGroupReq if err := json.Unmarshal(data, &req); err != nil { resp.RetMsg = err.Error() - s.Logger.Error("TruncateShardGroups fail", zap.Error(err)) + s.Logger.Error("DeleteShardGroup fail", zap.Error(err)) return } + req.Now = time.Now() err = s.ProposeAndWait(internal.DeleteShardGroup, data, nil) if err != nil { resp.RetMsg = err.Error() - s.Logger.Error("TruncateShardGroups fail", + s.Logger.Error("DeleteShardGroup fail", zap.String("Database", req.Database), zap.String("Policy", req.Policy), zap.Uint64("Id", req.Id), @@ -1044,7 +1084,7 @@ func (s *MetaService) DeleteShardGroup(w http.ResponseWriter, r *http.Request) { resp.RetCode = 0 resp.RetMsg = "ok" - s.Logger.Info("TruncateShardGroups ok", + s.Logger.Info("DeleteShardGroup ok", zap.String("Database", req.Database), zap.String("Policy", req.Policy), zap.Uint64("Id", req.Id)) @@ -1296,8 +1336,9 @@ func (s *MetaService) DropSubscription(w http.ResponseWriter, r *http.Request) { } type AcquireLeaseReq struct { - Name string - NodeId uint64 + Name string + NodeId uint64 + RequestTime int64 // timestamp in millis } type AcquireLeaseResp struct { @@ -1325,23 +1366,38 @@ func (s *MetaService) AcquireLease(w http.ResponseWriter, r *http.Request) { return } + // do pre-check before proposing + if !s.Node.ShouldTryAcquireLease(req.Name, req.NodeId) { + resp.RetMsg = ErrAcquiredByOther.Error() + return + } + + req.RequestTime = time.Now().UnixNano() / 1e6 + data, _ = json.Marshal(&req) + lease := &meta.Lease{} err = s.ProposeAndWait(internal.AcquireLease, data, lease) if err != nil { resp.RetMsg = err.Error() - s.Logger.Error("AcquireLease fail", - zap.String("Name", req.Name), - zap.Uint64("NodeId", req.NodeId), - zap.Error(err)) + if !strings.Contains(err.Error(), "another node has the lease") { + s.Logger.Error( + fmt.Sprintf( + "AcquireLease fail, name=%s, node=%d", + req.Name, req.NodeId, + ), + zap.Error(err), + ) + } return } resp.Lease = *lease resp.RetCode = 0 resp.RetMsg = "ok" - s.Logger.Info("AcquireLease ok", + s.Logger.Debug("AcquireLease ok", zap.String("Name", req.Name), - zap.Uint64("NodeId", req.NodeId)) + zap.Uint64("NodeId", req.NodeId), + ) } type DataResp struct { @@ -1364,6 +1420,15 @@ func (s *MetaService) Data(w http.ResponseWriter, r *http.Request) { } data := s.cli.Data() + // set metad peers + peers := s.Node.Transport.ClonePeers() + data.MetaNodes = nil + for id, addr := range peers { + data.MetaNodes = append(data.MetaNodes, meta.NodeInfo{ + ID: id, + Host: addr, + }) + } resp.Data, err = data.MarshalBinary() if err != nil { resp.RetMsg = err.Error() @@ -1374,6 +1439,46 @@ func (s *MetaService) Data(w http.ResponseWriter, r *http.Request) { resp.RetMsg = "ok" } +type FreezeDataNodeReq struct { + Id uint64 + Freeze bool +} +type FreezeDataNodeResp struct { + CommonResp +} + +func (s *MetaService) FreezeDataNode(w http.ResponseWriter, r *http.Request) { + resp := new(FreezeDataNodeResp) + resp.RetCode = -1 + resp.RetMsg = "fail" + defer WriteResp(w, &resp) + + data, err := ioutil.ReadAll(r.Body) + if err != nil { + resp.RetMsg = err.Error() + s.Logger.Error("FreezeDataNode fail", zap.Error(err)) + return + } + + var req FreezeDataNodeReq + if err := json.Unmarshal(data, &req); err != nil { + resp.RetMsg = err.Error() + s.Logger.Error("FreezeDataNode fail", zap.Error(err)) + return + } + + err = s.ProposeAndWait(internal.FreezeDataNode, data, nil) + if err != nil { + resp.RetMsg = err.Error() + s.Logger.Error(fmt.Sprintf("FreezeDataNode fail, id=%d, freeze=%t", req.Id, req.Freeze), zap.Error(err)) + return + } + + resp.RetCode = 0 + resp.RetMsg = "ok" + s.Logger.Info(fmt.Sprintf("FreezeDataNode ok, id=%d, freeze=%t", req.Id, req.Freeze)) +} + type PingResp struct { CommonResp Index uint64 @@ -1381,7 +1486,7 @@ type PingResp struct { func (s *MetaService) Ping(w http.ResponseWriter, r *http.Request) { resp := new(PingResp) - resp.Index = s.cli.Data().Index + resp.Index = s.cli.DataIndex() resp.RetCode = 0 resp.RetMsg = "ok" WriteResp(w, &resp) @@ -1407,6 +1512,7 @@ func initHttpHandler(s *MetaService) { http.HandleFunc(DROP_RETENTION_POLICY_PATH, s.DropRetentionPolicy) http.HandleFunc(DELETE_DATA_NODE_PATH, s.DeleteDataNode) + http.HandleFunc(FREEZE_DATA_NODE_PATH, s.FreezeDataNode) http.HandleFunc(CREATE_RETENTION_POLICY_PATH, s.CreateRetentionPolicy) http.HandleFunc(UPDATE_RETENTION_POLICY_PATH, s.UpdateRetentionPolicy) http.HandleFunc(CREATE_USER_PATH, s.CreateUser) @@ -1416,6 +1522,7 @@ func initHttpHandler(s *MetaService) { http.HandleFunc(SET_ADMIN_PRIVILEGE, s.SetAdminPrivilege) http.HandleFunc(AUTHENTICATE_PATH, s.Authenticate) http.HandleFunc(DROP_SHARD_PATH, s.DropShard) + http.HandleFunc(ADD_SHARD_OWNER, s.AddShardOwner) http.HandleFunc(REMOVE_SHARD_OWNER, s.RemoveShardOwner) http.HandleFunc(TRUNCATE_SHARD_GROUPS_PATH, s.TruncateShardGroups) http.HandleFunc(PRUNE_SHARD_GROUPS_PATH, s.PruneShardGroups) diff --git a/raftmeta/meta_client.go b/raftmeta/meta_store.go similarity index 81% rename from raftmeta/meta_client.go rename to raftmeta/meta_store.go index ab0cbeb..343e40a 100644 --- a/raftmeta/meta_client.go +++ b/raftmeta/meta_store.go @@ -1,13 +1,14 @@ package raftmeta import ( + "time" + imeta "github.com/angopher/chronus/services/meta" "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxql" - "time" ) -type MetaClient interface { +type MetaStore interface { MarshalBinary() ([]byte, error) ReplaceData(data *imeta.Data) error Data() imeta.Data @@ -17,12 +18,15 @@ type MetaClient interface { CreateRetentionPolicy(database string, spec *meta.RetentionPolicySpec, makeDefault bool) (*meta.RetentionPolicyInfo, error) CreateShardGroup(database, policy string, timestamp time.Time) (*meta.ShardGroupInfo, error) CreateSubscription(database, rp, name, mode string, destinations []string) error - CreateUser(name, password string, admin bool) (meta.User, error) + CreateUser(name, hashedPassword string, admin bool) (meta.User, error) CreateDataNode(httpAddr, tcpAddr string) (*meta.NodeInfo, error) DeleteDataNode(id uint64) error + IsDataNodeFreezed(id uint64) bool + FreezeDataNode(id uint64) error + UnfreezeDataNode(id uint64) error Authenticate(username, password string) (meta.User, error) - PruneShardGroups() error - DeleteShardGroup(database, policy string, id uint64) error + PruneShardGroups(expiration time.Time) error + DeleteShardGroup(database, policy string, id uint64, t time.Time) error PrecreateShardGroups(from, to time.Time) error AddShardOwner(shardID, nodeID uint64) error @@ -37,5 +41,5 @@ type MetaClient interface { SetPrivilege(username, database string, p influxql.Privilege) error TruncateShardGroups(t time.Time) error UpdateRetentionPolicy(database, name string, rpu *meta.RetentionPolicyUpdate, makeDefault bool) error - UpdateUser(name, password string) error + UpdateUser(name, hashedPassword string) error } diff --git a/raftmeta/node.go b/raftmeta/node.go index 887b7a5..476b7fe 100644 --- a/raftmeta/node.go +++ b/raftmeta/node.go @@ -5,23 +5,25 @@ import ( "encoding/json" "errors" "fmt" + "io" + "math/rand" + "os" + "sync" + "sync/atomic" + "time" + "github.com/angopher/chronus/raftmeta/internal" imeta "github.com/angopher/chronus/services/meta" "github.com/angopher/chronus/x" - "github.com/coreos/etcd/pkg/wait" - "github.com/coreos/etcd/raft" - "github.com/coreos/etcd/raft/raftpb" - "github.com/dgraph-io/badger" - "github.com/dgraph-io/badger/options" + "github.com/dgraph-io/badger/v2" + "github.com/dgraph-io/badger/v2/options" "github.com/dgraph-io/dgraph/raftwal" "github.com/influxdata/influxdb/services/meta" + "go.etcd.io/etcd/pkg/wait" + "go.etcd.io/etcd/raft" + "go.etcd.io/etcd/raft/raftpb" "go.uber.org/zap" "golang.org/x/net/trace" - "math/rand" - "os" - "sync" - "sync/atomic" - "time" ) var errInternalRetry = errors.New("Retry Raft proposal internally") @@ -121,9 +123,9 @@ type RaftNode struct { ID uint64 Node raft.Node - MetaCli MetaClient + MetaStore MetaStore //用于Continuous query - leases *meta.Leases + leases *ClusterLeases //raft集群内部配置状态 RaftConfState *raftpb.ConfState @@ -139,7 +141,8 @@ type RaftNode struct { Config Config //用于存储raft日志和snapshot - Storage *raftwal.DiskStorage + Storage *raftwal.DiskStorage + walStore *badger.DB //节点之间的通信模块 Transport interface { @@ -173,25 +176,32 @@ type RaftNode struct { //only for test ApplyCallBack func(proposal *internal.Proposal, index uint64) - Logger *zap.Logger + Logger *zap.Logger + SugaredLogger *zap.SugaredLogger } -func NewRaftNode(config Config) *RaftNode { +func NewRaftNode(config Config, logger *zap.Logger) *RaftNode { c := &raft.Config{ ID: config.RaftId, ElectionTick: config.ElectionTick, HeartbeatTick: config.HeartbeatTick, MaxSizePerMsg: config.MaxSizePerMsg, MaxInflightMsgs: config.MaxInflightMsgs, + Logger: newRaftLoggerBridge(logger), } - walDir := config.WalDir - x.Checkf(os.MkdirAll(walDir, 0700), "Error while creating WAL dir.") - kvOpt := badger.DefaultOptions + x.Checkf(os.MkdirAll(config.WalDir, 0700), "Error while creating WAL dir.") + kvOpt := badger.DefaultOptions(config.WalDir) kvOpt.SyncWrites = true - kvOpt.Dir = config.WalDir - kvOpt.ValueDir = config.WalDir + kvOpt.Logger = NewBadgerLoggerBridge(logger) kvOpt.TableLoadingMode = options.MemoryMap + kvOpt.ValueLogFileSize = 8 << 20 + kvOpt.MaxTableSize = 8 << 20 + kvOpt.NumLevelZeroTables = 2 + kvOpt.NumMemtables = 4 + kvOpt.LevelSizeMultiplier = 5 + kvOpt.LevelOneSize = 32 << 20 + kvOpt.NumCompactors = 2 walStore, err := badger.Open(kvOpt) x.Checkf(err, "Error while creating badger KV WAL store") @@ -210,13 +220,17 @@ func NewRaftNode(config Config) *RaftNode { //storage := raft.NewMemoryStorage() storage := raftwal.Init(walStore, c.ID, 0) c.Storage = storage + selfLogger := logger.With(zap.String("raftmeta", "RaftNode")) return &RaftNode{ - leases: meta.NewLeases(meta.DefaultLeaseDuration), + leases: NewClusterLeases(meta.DefaultLeaseDuration, selfLogger), ID: c.ID, RaftConfig: c, + Logger: selfLogger, + SugaredLogger: selfLogger.Sugar(), Config: config, RaftCtx: rc, Storage: storage, + walStore: walStore, Done: make(chan struct{}), props: newProposals(), rand: rand.New(&lockedSource{src: rand.NewSource(time.Now().UnixNano())}), @@ -228,8 +242,81 @@ func NewRaftNode(config Config) *RaftNode { } } -func (s *RaftNode) WithLogger(log *zap.Logger) { - s.Logger = log.With(zap.String("raftmeta", "RaftNode")) +func (s *RaftNode) resetPeersInSnapshot(snapshot *raftpb.Snapshot) error { + var sndata internal.SnapshotData + err := json.Unmarshal(snapshot.Data, &sndata) + if err != nil { + return err + } + sndata.PeersAddr = make(map[uint64]string) + for _, p := range s.Config.Peers { + sndata.PeersAddr[p.RaftId] = p.Addr + } + snapshot.Data, err = json.Marshal(&sndata) + if err != nil { + return err + } + return nil +} + +func (s *RaftNode) Restore(filePath string) error { + f, err := os.OpenFile(filePath, os.O_RDONLY, 0) + if err != nil { + return err + } + defer f.Close() + info, err := os.Lstat(filePath) + if err != nil { + return err + } + s.Logger.Warn("Restore from snapfile", zap.String("file", filePath), zap.Int64("size", info.Size())) + snapdata := make([]byte, info.Size()) + _, err = io.ReadFull(f, snapdata) + if err != nil { + return err + } + snapshot := raftpb.Snapshot{} + err = json.Unmarshal(snapdata, &snapshot) + if err != nil { + return err + } + s.Logger.Warn(fmt.Sprintf( + "Term=%d, Index=%d", + snapshot.Metadata.Term, + snapshot.Metadata.Index, + )) + if s.RaftConfState == nil { + return errors.New("ConfState has not been set yet") + } + snapshot.Metadata.ConfState = *s.RaftConfState + s.Logger.Warn(fmt.Sprintf( + "Voters=%v, Learners=%v", + snapshot.Metadata.ConfState.Voters, + snapshot.Metadata.ConfState.Learners, + )) + err = s.resetPeersInSnapshot(&snapshot) + if err != nil { + return err + } + return s.Storage.Save(raftpb.HardState{}, []raftpb.Entry{}, snapshot) +} + +func (s *RaftNode) Dump(filePath string) error { + f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_CREATE, 0644) + if err != nil { + return err + } + defer f.Close() + sp, err := s.Storage.Snapshot() + if err != nil { + return err + } + data, err := json.Marshal(&sp) + if err != nil { + return err + } + _, err = f.Write(data) + return err } // uniqueKey is meant to be unique across all the replicas. @@ -264,14 +351,14 @@ func (s *RaftNode) leaderChangedNotify() <-chan struct{} { return s.leaderChanged } -func (s *RaftNode) restoreFromSnapshot() { +func (s *RaftNode) restoreFromSnapshot() bool { s.Logger.Info("restore from snapshot") sp, err := s.Storage.Snapshot() x.Checkf(err, "Unable to get existing snapshot") if raft.IsEmptySnap(sp) { s.Logger.Info("empty snapshot. ignore") - return + return false } s.SetConfState(&sp.Metadata.ConfState) s.setAppliedIndex(sp.Metadata.Index) @@ -286,12 +373,34 @@ func (s *RaftNode) restoreFromSnapshot() { err = metaData.UnmarshalBinary(sndata.Data) x.Checkf(err, "meta data UnmarshalBinary fail") - err = s.MetaCli.ReplaceData(metaData) + err = s.MetaStore.ReplaceData(metaData) x.Checkf(err, "meta cli ReplaceData fail") + + return true +} + +func (s *RaftNode) resetPeersFromConfig() { + for _, peer := range s.Config.Peers { + s.Transport.SetPeer(peer.RaftId, peer.Addr) + } +} + +func (s *RaftNode) reclaimDiskSpace() { + ticker := time.NewTicker(5 * time.Minute) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + s.walStore.RunValueLogGC(0.5) + case <-s.Done: + return + } + } } func (s *RaftNode) InitAndStartNode() { - peers := []raft.Peer{} + peers := make([]raft.Peer, 0, len(s.Config.Peers)) for _, p := range s.Config.Peers { rc := internal.RaftContext{Addr: p.Addr, ID: p.RaftId} data, err := json.Marshal(&rc) @@ -305,8 +414,11 @@ func (s *RaftNode) InitAndStartNode() { if restart { s.Logger.Info("Restarting node") - s.restoreFromSnapshot() + restored := s.restoreFromSnapshot() s.Node = raft.RestartNode(s.RaftConfig) + if !restored { + s.resetPeersFromConfig() + } } else { s.Logger.Info("Starting node") if len(peers) == 0 { @@ -315,7 +427,7 @@ func (s *RaftNode) InitAndStartNode() { x.Check(err) s.Node = raft.StartNode(s.RaftConfig, []raft.Peer{{ID: s.ID, Context: data}}) } else { - rpeers := make([]raft.Peer, 0) + rpeers := make([]raft.Peer, 0, len(s.Config.Peers)) for _, peer := range s.Config.Peers { rpeers = append(rpeers, raft.Peer{ID: uint64(peer.RaftId)}) } @@ -323,13 +435,7 @@ func (s *RaftNode) InitAndStartNode() { //x.Checkf(err, "join peers fail") //s.Logger.Info("join peers success") s.Node = raft.StartNode(s.RaftConfig, rpeers) - - for _, peer := range s.Config.Peers { - if peer.RaftId != s.ID { - s.Transport.SetPeer(peer.RaftId, peer.Addr) - } - } - + s.resetPeersFromConfig() } } } @@ -357,6 +463,8 @@ func (s *RaftNode) Run() { t := time.NewTicker(time.Duration(s.Config.TickTimeMs) * time.Millisecond) defer t.Stop() + go s.reclaimDiskSpace() + var leader uint64 for { @@ -364,8 +472,10 @@ func (s *RaftNode) Run() { case <-snapshotTicker.C: if leader == s.ID { go func() { - err := s.trigerSnapshot() - s.Logger.Error("calculateSnapshot fail", zap.Error(err)) + err := s.triggerSnapshot() + if err != nil { + s.Logger.Error("calculateSnapshot fail", zap.Error(err)) + } }() } case <-checkSumTicker.C: @@ -411,7 +521,7 @@ func (s *RaftNode) Run() { s.applyCh <- ew } for _, entry := range rd.CommittedEntries { - s.Logger.Info("process entry", zap.Uint64("term", entry.Term), zap.Uint64("index", entry.Index), zap.String("type", entry.Type.String())) + s.Logger.Debug("process entry", zap.Uint64("term", entry.Term), zap.Uint64("index", entry.Index), zap.String("type", entry.Type.String())) ew := &internal.EntryWrapper{Entry: entry, Restore: false} s.applyCh <- ew } @@ -429,7 +539,7 @@ func (s *RaftNode) Run() { //TODO:optimize func (s *RaftNode) triggerChecksum() { - s.Logger.Info("trigger check sum") + s.Logger.Info("trigger checksum") if s.lastChecksum.needVerify { go func() { @@ -462,8 +572,8 @@ func (s *RaftNode) triggerChecksum() { } } -func (s *RaftNode) trigerSnapshot() error { - s.Logger.Info("trigerSnapshot") +func (s *RaftNode) triggerSnapshot() error { + s.Logger.Info("triggerSnapshot") var sn internal.CreateSnapshot //_, err = s.Storage.LastIndex() //x.Check(err) @@ -506,7 +616,7 @@ func (s *RaftNode) processApplyCh() { x.Fatalf("Unable to unmarshal proposal: %v %q\n", err, e.Data) } err := s.applyCommitted(proposal, e.Index) - s.Logger.Info("Applied proposal", zap.String("key", proposal.Key), zap.Uint64("index", e.Index), zap.Error(err)) + s.Logger.Debug("Applied proposal", zap.String("key", proposal.Key), zap.Uint64("index", e.Index), zap.Error(err)) s.props.Done(proposal.Key, err) } @@ -561,12 +671,15 @@ func (s *RaftNode) ProposeConfChange(ctx context.Context, cc raftpb.ConfChange) } func (s *RaftNode) ProposeAndWait(ctx context.Context, proposal *internal.Proposal, retData interface{}) error { - cctx, cancel := context.WithTimeout(ctx, 5*time.Second) - defer cancel() - che := make(chan error, 1) + if _, ok := ctx.Deadline(); !ok { + // introduce timeout if needed + var cancel context.CancelFunc + ctx, cancel = context.WithTimeout(ctx, 5*time.Second) + defer cancel() + } pctx := &proposalCtx{ - ch: che, - ctx: cctx, + ch: make(chan error, 1), + ctx: ctx, retData: retData, } key := s.uniqueKey() @@ -581,7 +694,7 @@ func (s *RaftNode) ProposeAndWait(ctx context.Context, proposal *internal.Propos data, err := json.Marshal(proposal) x.Check(err) - if err = s.Propose(cctx, data); err != nil { + if err = s.Propose(ctx, data); err != nil { return x.Wrapf(err, "While proposing") } @@ -590,22 +703,21 @@ func (s *RaftNode) ProposeAndWait(ctx context.Context, proposal *internal.Propos } select { - case err = <-che: + case err = <-pctx.ch: // We arrived here by a call to n.props.Done(). if tr, ok := trace.FromContext(ctx); ok { tr.LazyPrintf("Done with error: %v", err) } return err case <-ctx.Done(): - if tr, ok := trace.FromContext(ctx); ok { - tr.LazyPrintf("External context timed out with error: %v.", ctx.Err()) + if ctx.Err() == context.DeadlineExceeded { + if tr, ok := trace.FromContext(ctx); ok { + tr.LazyPrintf("Propose timed out.") + } + return errInternalRetry } + return ctx.Err() - case <-cctx.Done(): - if tr, ok := trace.FromContext(ctx); ok { - tr.LazyPrintf("Internal context timed out with error: %v. Retrying...", cctx.Err()) - } - return errInternalRetry } } @@ -643,3 +755,7 @@ func (s *RaftNode) PastLife() (idx uint64, restart bool, rerr error) { } return } + +func (s *RaftNode) ShouldTryAcquireLease(name string, nodeId uint64) bool { + return s.leases.ShouldTry(name, nodeId) +} diff --git a/raftmeta/path.go b/raftmeta/path.go index 1547417..ceb1dc3 100644 --- a/raftmeta/path.go +++ b/raftmeta/path.go @@ -7,6 +7,7 @@ const ( CREATE_DATA_NODE_PATH = "/create_data_node" DROP_RETENTION_POLICY_PATH = "/drop_retention_policy" DELETE_DATA_NODE_PATH = "/delete_data_node" + FREEZE_DATA_NODE_PATH = "/freeze_data_node" CREATE_RETENTION_POLICY_PATH = "/create_retention_policy" UPDATE_RETENTION_POLICY_PATH = "/update_retention_policy" CREATE_USER_PATH = "/create_user" diff --git a/raftmeta/raft_logger_bridge.go b/raftmeta/raft_logger_bridge.go new file mode 100644 index 0000000..a3c04c5 --- /dev/null +++ b/raftmeta/raft_logger_bridge.go @@ -0,0 +1,64 @@ +package raftmeta + +import ( + "fmt" + + "go.etcd.io/etcd/raft" + "go.uber.org/zap" +) + +type raftLoggerBridge struct { + raft.Logger + suger *zap.SugaredLogger +} + +func newRaftLoggerBridge(logger *zap.Logger) *raftLoggerBridge { + if logger == nil { + logger = zap.NewNop() + } + return &raftLoggerBridge{ + suger: logger.Sugar(), + } +} + +func (b *raftLoggerBridge) Debug(v ...interface{}) { + b.suger.Debug(v) +} +func (b *raftLoggerBridge) Debugf(format string, v ...interface{}) { + b.suger.Debug(fmt.Sprintf(format, v...)) +} + +func (b *raftLoggerBridge) Error(v ...interface{}) { + b.suger.Error(v) +} +func (b *raftLoggerBridge) Errorf(format string, v ...interface{}) { + b.suger.Error(fmt.Sprintf(format, v...)) +} + +func (b *raftLoggerBridge) Info(v ...interface{}) { + b.suger.Info(v) +} +func (b *raftLoggerBridge) Infof(format string, v ...interface{}) { + b.suger.Info(fmt.Sprintf(format, v...)) +} + +func (b *raftLoggerBridge) Warning(v ...interface{}) { + b.suger.Warn(v) +} +func (b *raftLoggerBridge) Warningf(format string, v ...interface{}) { + b.suger.Warn(fmt.Sprintf(format, v...)) +} + +func (b *raftLoggerBridge) Fatal(v ...interface{}) { + b.suger.Fatal(v) +} +func (b *raftLoggerBridge) Fatalf(format string, v ...interface{}) { + b.suger.Fatal(fmt.Sprintf(format, v...)) +} + +func (b *raftLoggerBridge) Panic(v ...interface{}) { + b.suger.Panic(v) +} +func (b *raftLoggerBridge) Panicf(format string, v ...interface{}) { + b.suger.Panic(fmt.Sprintf(format, v...)) +} diff --git a/raftmeta/raft_test.go b/raftmeta/raft_test.go index 8329160..e89fe01 100644 --- a/raftmeta/raft_test.go +++ b/raftmeta/raft_test.go @@ -7,10 +7,10 @@ import ( "os" "testing" - "github.com/coreos/etcd/raft" - "github.com/coreos/etcd/raft/raftpb" "github.com/influxdata/influxdb/logger" "github.com/influxdata/influxdb/services/meta" + "go.etcd.io/etcd/raft" + "go.etcd.io/etcd/raft/raftpb" "github.com/angopher/chronus/raftmeta" "github.com/angopher/chronus/raftmeta/internal" @@ -126,10 +126,9 @@ func newService(config raftmeta.Config, t *fakeTransport, cb func(proposal *inte x.Check(err) log := logger.New(os.Stderr) - node := raftmeta.NewRaftNode(config) - node.MetaCli = metaCli + node := raftmeta.NewRaftNode(config, log) + node.MetaStore = metaCli node.ApplyCallBack = cb - node.WithLogger(log) node.Transport = t node.InitAndStartNode() diff --git a/raftmeta/transport.go b/raftmeta/transport.go index 0c6f2fa..8d8c342 100644 --- a/raftmeta/transport.go +++ b/raftmeta/transport.go @@ -5,17 +5,39 @@ import ( "context" "encoding/json" "fmt" - "github.com/angopher/chronus/raftmeta/internal" - "github.com/angopher/chronus/x" - "github.com/coreos/etcd/raft" - "github.com/coreos/etcd/raft/raftpb" - "go.uber.org/zap" "io/ioutil" "net" "net/http" "strconv" + "strings" "sync" "time" + + "github.com/angopher/chronus/raftmeta/internal" + "github.com/angopher/chronus/x" + "go.etcd.io/etcd/raft" + "go.etcd.io/etcd/raft/raftpb" + "go.uber.org/zap" + "golang.org/x/time/rate" +) + +var ( + loggingLimiter = rate.NewLimiter(1, 1) + httpClient = http.Client{ + Transport: &http.Transport{ + Proxy: http.ProxyFromEnvironment, + DialContext: (&net.Dialer{ + Timeout: 10 * time.Second, + KeepAlive: 30 * time.Second, + DualStack: true, + }).DialContext, + ForceAttemptHTTP2: true, + MaxIdleConns: 100, + IdleConnTimeout: 90 * time.Second, + TLSHandshakeTimeout: 10 * time.Second, + ExpectContinueTimeout: 1 * time.Second, + }, + } ) type Transport struct { @@ -87,7 +109,7 @@ func (t *Transport) SendMessage(messages []raftpb.Message) { } url := fmt.Sprintf("http://%s/message", addr) err = Request(url, data) - if err != nil { + if err != nil && loggingLimiter.Allow() { t.Logger.Error("Request fail:", zap.Error(err), zap.String("url", url)) } } @@ -117,14 +139,17 @@ func (s *RaftNode) HandleUpdateCluster(w http.ResponseWriter, r *http.Request) { resp := &CommonResp{} resp.RetCode = -1 resp.RetMsg = "fail" + w.Header().Set("Content-Type", "application/json; charset=utf-8") defer WriteResp(w, &resp) var err error + peers := s.Transport.ClonePeers() typ := raftpb.ConfChangeAddNode data := []byte{} var nodeId uint64 op := r.FormValue("op") - if op == "add" || op == "update" { + switch op { + case "add", "update": data, err = ioutil.ReadAll(r.Body) if err != nil { resp.RetMsg = err.Error() @@ -134,11 +159,37 @@ func (s *RaftNode) HandleUpdateCluster(w http.ResponseWriter, r *http.Request) { err = json.Unmarshal(data, &rc) x.Check(err) nodeId = rc.ID - } else if op == "remove" { - typ = raftpb.ConfChangeRemoveNode + + // check addr + for _, addr := range peers { + if addr == rc.Addr { + resp.RetMsg = "specified node address already exists" + return + } + } + + // check node id + switch op { + case "add": + if _, ok := peers[nodeId]; ok { + resp.RetMsg = fmt.Sprintf("specified node id already exists: %d", nodeId) + return + } + case "update": + if _, ok := peers[nodeId]; !ok { + resp.RetMsg = fmt.Sprintf("specified node id doesn't exist: %d", nodeId) + return + } + } + case "remove": nodeId, err = strconv.ParseUint(r.FormValue("node_id"), 10, 64) + if _, ok := peers[nodeId]; !ok { + resp.RetMsg = fmt.Sprintf("unkown node id: %d", nodeId) + return + } + typ = raftpb.ConfChangeRemoveNode x.Check(err) - } else { + default: resp.RetMsg = fmt.Sprintf("unkown op:%s", op) return } @@ -159,8 +210,95 @@ func (s *RaftNode) HandleUpdateCluster(w http.ResponseWriter, r *http.Request) { resp.RetMsg = err.Error() return } + if op == "add" { + // trigger a new snapshot, make sure the new node is existed in snapshot sent later. + s.triggerSnapshot() + } + resp.RetCode = 0 + resp.RetMsg = "ok" +} + +func (s *RaftNode) HandleStatusNode(w http.ResponseWriter, r *http.Request) { + resp := &StatusNodeResp{} + resp.RetCode = -1 + resp.RetMsg = "fail" + w.Header().Set("Content-Type", "application/json; charset=utf-8") + defer WriteResp(w, &resp) + + peers := s.Transport.ClonePeers() + status := s.Node.Status() + resp.Status.ID = status.ID + resp.Status.Vote = status.Vote + resp.Status.Match = status.Applied + resp.Status.Next = resp.Status.Match + 1 + resp.Status.Role = strings.Replace(status.RaftState.String(), "State", "", 1) + if addr, ok := peers[status.ID]; ok { + resp.Status.Addr = addr + } + resp.RetCode = 0 + resp.RetMsg = "ok" +} + +func (s *RaftNode) HandleStatusCluster(w http.ResponseWriter, r *http.Request) { + resp := &StatusClusterResp{} + resp.RetCode = -1 + resp.RetMsg = "fail" + w.Header().Set("Content-Type", "application/json; charset=utf-8") + defer WriteResp(w, &resp) + + peers := s.Transport.ClonePeers() + leader := s.Node.Status().Lead + if leader != s.ID { + // forward + if r.URL.Query().Get("forward") == "1" { + resp.RetMsg = "Error forwarding" + return + } + if addr, ok := peers[leader]; ok { + r, err := forwardStatusCluster(addr) + if err != nil { + resp.RetMsg = err.Error() + } else { + resp = r + } + } + return + } + nodes := make([]NodeStatus, 0, len(peers)) + status := s.Node.Status() + prs := status.Progress + for id, addr := range peers { + idx := len(nodes) + nodes = append(nodes, NodeStatus{ + ID: id, + Addr: addr, + Match: 0, + Next: 0, + Role: "Unreachable", + Progress: "Unreachable", + }) + + nodeStatus, err := statusPeer(addr) + if err != nil { + continue + } + nodes[idx].Role = nodeStatus.Role + nodes[idx].Vote = nodeStatus.Vote + + if pr, ok := prs[id]; ok { + nodes[idx].Match = pr.Match + nodes[idx].Next = pr.Next + nodes[idx].Progress = strings.Replace(pr.State.String(), "ProgressState", "", 1) + } + } + resp.RetCode = 0 resp.RetMsg = "ok" + resp.Nodes = nodes + resp.Applied = status.Applied + resp.Commit = status.Commit + resp.Leader = status.Lead + resp.Term = status.Term } func (s *RaftNode) HandleMessage(w http.ResponseWriter, r *http.Request) { @@ -176,45 +314,67 @@ func (s *RaftNode) HandleMessage(w http.ResponseWriter, r *http.Request) { err = msg.Unmarshal(data) x.Check(err) if msg.Type != raftpb.MsgHeartbeat && msg.Type != raftpb.MsgHeartbeatResp { - s.Logger.Info("recv message", zap.String("type", msg.Type.String())) + s.Logger.Debug("recv message", zap.String("type", msg.Type.String())) } s.RecvRaftRPC(context.Background(), msg) } -func Request(url string, data []byte) error { +func postRequest(url string, data []byte) ([]byte, error) { req, err := http.NewRequest("POST", url, bytes.NewReader(data)) if err != nil { - return err + return nil, err } req.Header.Set("Content-Type", "application/json") - req.Header.Set("Connection", "close") - - client := http.Client{ - Transport: &http.Transport{ - Dial: func(netw, addr string) (net.Conn, error) { - deadline := time.Now().Add(10 * time.Second) //TODO: timeout from config - c, err := net.DialTimeout(netw, addr, time.Second) - if err != nil { - return nil, err - } - c.SetDeadline(deadline) - return c, nil - }, - }, - } - res, err := client.Do(req) + res, err := httpClient.Do(req) if err != nil { - return err + return nil, err } defer res.Body.Close() resData, err := ioutil.ReadAll(res.Body) + if err != nil { + return nil, err + } + return resData, nil +} + +func forwardStatusCluster(addr string) (*StatusClusterResp, error) { + data, err := postRequest(fmt.Sprint("http://", addr, "/status_cluster?forward=1"), nil) + if err != nil { + return nil, err + } + + resp := &StatusClusterResp{} + resp.RetCode = -1 + resp.RetMsg = "fail" + err = json.Unmarshal(data, resp) + return resp, nil +} + +func statusPeer(addr string) (*NodeStatus, error) { + data, err := postRequest(fmt.Sprint("http://", addr, "/status_node"), nil) + if err != nil { + return nil, err + } + + resp := &StatusNodeResp{} + resp.RetCode = -1 + resp.RetMsg = "fail" + err = json.Unmarshal(data, resp) + if resp.RetCode != 0 { + return nil, fmt.Errorf("fail. err:%s", resp.RetMsg) + } + return &resp.Status, nil +} + +func Request(url string, data []byte) error { + data, err := postRequest(url, data) if err != nil { return err } resp := &CommonResp{RetCode: -1} - err = json.Unmarshal(resData, resp) + err = json.Unmarshal(data, resp) if resp.RetCode != 0 { return fmt.Errorf("fail. err:%s", resp.RetMsg) } diff --git a/services/controller/service.go b/services/controller/service.go index 9f1b52f..d9bdcc1 100644 --- a/services/controller/service.go +++ b/services/controller/service.go @@ -3,23 +3,28 @@ package controller import ( "encoding/json" + "errors" "fmt" "io" "net" + "sort" "strings" "sync" "time" "github.com/influxdata/influxdb" "github.com/influxdata/influxdb/services/meta" + "github.com/influxdata/influxdb/tsdb" "go.uber.org/zap" "github.com/angopher/chronus/coordinator" + "github.com/angopher/chronus/services/migrate" ) const ( // MuxHeader is the header byte used for the TCP muxer. - MuxHeader = 4 + MuxHeader = 4 + MILLISECOND = 1e6 ) type Service struct { @@ -30,29 +35,38 @@ type Service struct { MetaClient interface { TruncateShardGroups(t time.Time) error DeleteDataNode(id uint64) error + IsDataNodeFreezed(id uint64) bool + FreezeDataNode(id uint64) error + UnfreezeDataNode(id uint64) error DataNodeByTCPHost(addr string) (*meta.NodeInfo, error) RemoveShardOwner(shardID, nodeID uint64) error DataNodes() ([]meta.NodeInfo, error) + RetentionPolicy(database, name string) (*meta.RetentionPolicyInfo, error) + Databases() []meta.DatabaseInfo + + ShardOwner(shardID uint64) (database, policy string, sgi *meta.ShardGroupInfo) + AddShardOwner(shardID, nodeID uint64) error } TSDBStore interface { + Path() string + ShardRelativePath(id uint64) (string, error) + CreateShard(database, retentionPolicy string, shardID uint64, enabled bool) error DeleteShard(id uint64) error + Shard(id uint64) *tsdb.Shard } Listener net.Listener Logger *zap.Logger - ShardCopier interface { - CopyShard(sourceAddr string, shardId uint64) error - Query() []CopyShardTask - Kill(shardId uint64, source, destination string) - } + migrateManager *migrate.Manager } // NewService returns a new instance of Service. func NewService(c Config) *Service { return &Service{ - Logger: zap.NewNop(), + Logger: zap.NewNop(), + migrateManager: migrate.NewManager(c.MaxShardCopyTasks), } } @@ -61,6 +75,7 @@ func (s *Service) Open() error { s.Logger.Info("Starting controller service") s.wg.Add(1) + s.migrateManager.Start() go s.serve() return nil } @@ -72,6 +87,7 @@ func (s *Service) Close() error { return err } } + s.migrateManager.Close() s.wg.Wait() return nil } @@ -79,6 +95,7 @@ func (s *Service) Close() error { // WithLogger sets the logger on the service. func (s *Service) WithLogger(log *zap.Logger) { s.Logger = log.With(zap.String("service", "controller")) + s.migrateManager.WithLogger(log.With(zap.String("service", "migrate_manager"))) } // serve serves snapshot requests from the listener. @@ -95,7 +112,7 @@ func (s *Service) serve() { s.Logger.Info("Error accepting snapshot request", zap.Error(err)) continue } - s.Logger.Info("accept new conn.") + s.Logger.Debug("accept new conn.") // Handle connection in separate goroutine. s.wg.Add(1) @@ -139,20 +156,26 @@ func (s *Service) handleConn(conn net.Conn) error { case RequestShowDataNodes: nodes, err := s.handleShowDataNodes() s.showDataNodesResponse(conn, nodes, err) + case RequestShards: + groupInfo, err := s.handleShards(conn) + s.shardsResponse(conn, groupInfo, err) + case RequestNodeShards: + shards, err := s.handleNodeShards(conn) + s.nodeShardsResponse(conn, shards, err) + case RequestShard: + db, rp, info, groupInfo, err := s.handleShard(conn) + s.shardResponse(conn, db, rp, info, groupInfo, err) + case RequestFreezeDataNode: + err = s.handleFreezeDataNode(conn) + s.freezeDataNodeResponse(conn, err) } return nil } func (s *Service) handleTruncateShard(conn net.Conn) error { - buf, err := coordinator.ReadLV(conn) - if err != nil { - s.Logger.Error("unable to read length-value", zap.Error(err)) - return err - } - var req TruncateShardRequest - if err := json.Unmarshal(buf, &req); err != nil { + if err := s.readRequest(conn, &req); err != nil { return err } @@ -188,19 +211,12 @@ func (s *Service) truncateShardResponse(w io.Writer, e error) { } func (s *Service) handleCopyShard(conn net.Conn) error { - buf, err := coordinator.ReadLV(conn) - if err != nil { - s.Logger.Error("unable to read length-value", zap.Error(err)) - return err - } - var req CopyShardRequest - if err := json.Unmarshal(buf, &req); err != nil { + if err := s.readRequest(conn, &req); err != nil { return err } - s.ShardCopier.CopyShard(req.SourceNodeAddr, req.ShardID) - return nil + return s.copyShard(req.SourceNodeAddr, req.ShardID) } func (s *Service) copyShardResponse(w io.Writer, e error) { @@ -227,81 +243,82 @@ func (s *Service) copyShardResponse(w io.Writer, e error) { } } -func (s *Service) handleCopyShardStatus(conn net.Conn) []CopyShardTask { - return s.ShardCopier.Query() +func toCopyTask(t *migrate.Task) CopyShardTask { + task := CopyShardTask{} + task.CurrentSize = t.Copied + task.Database = t.Database + task.Rp = t.Retention + task.ShardID = t.ShardId + task.Source = t.SrcHost + return task } -func (s *Service) copyShardStatusResponse(w io.Writer, tasks []CopyShardTask) { - // Build response. - var resp CopyShardStatusResponse - resp.Code = 0 - resp.Msg = "ok" - resp.Tasks = tasks - +func (s *Service) writeResponse(w io.Writer, t ResponseType, obj interface{}) { // Marshal response to binary. - buf, err := json.Marshal(&resp) + buf, err := json.Marshal(obj) if err != nil { - s.Logger.Error("error marshalling show copy shard response", zap.Error(err)) + s.Logger.Error(fmt.Sprint("Marshal fail: type=", t, ", error=", err.Error())) return } // Write to connection. - if err := coordinator.WriteTLV(w, byte(ResponseCopyShardStatus), buf); err != nil { - s.Logger.Error("show copy shard WriteTLV fail", zap.Error(err)) + if err := coordinator.WriteTLV(w, byte(t), buf); err != nil { + s.Logger.Error(fmt.Sprint("WriteTLV fail: type=", t, ", error=", err.Error())) } } -func (s *Service) handleKillCopyShard(conn net.Conn) error { - buf, err := coordinator.ReadLV(conn) +func (s *Service) readRequest(conn net.Conn, obj interface{}) error { + buf, err := coordinator.ReadLV(conn, 10*time.Second) if err != nil { s.Logger.Error("unable to read length-value", zap.Error(err)) return err } + if err := json.Unmarshal(buf, obj); err != nil { + return err + } + return nil +} + +func (s *Service) handleCopyShardStatus(conn net.Conn) []CopyShardTask { + tasks := s.migrateManager.Tasks() + result := make([]CopyShardTask, len(tasks)) + for i, t := range tasks { + result[i] = toCopyTask(t) + } + return result +} + +func (s *Service) copyShardStatusResponse(w io.Writer, tasks []CopyShardTask) { + // Build response. + var resp CopyShardStatusResponse + resp.Code = 0 + resp.Msg = "ok" + resp.Tasks = tasks + s.writeResponse(w, ResponseCopyShardStatus, &resp) +} + +func (s *Service) handleKillCopyShard(conn net.Conn) error { var req KillCopyShardRequest - if err := json.Unmarshal(buf, &req); err != nil { + if err := s.readRequest(conn, &req); err != nil { return err } - s.ShardCopier.Kill(req.ShardID, req.SourceNodeAddr, req.DestNodeAddr) + s.migrateManager.Kill(req.ShardID, req.SourceNodeAddr) return nil } func (s *Service) killCopyShardResponse(w io.Writer, e error) { // Build response. var resp KillCopyShardResponse - if e != nil { - resp.Code = 1 - resp.Msg = e.Error() - } else { - resp.Code = 0 - resp.Msg = "ok" - } - - // Marshal response to binary. - buf, err := json.Marshal(&resp) - if err != nil { - s.Logger.Error("error marshalling show copy shard response", zap.Error(err)) - return - } - - // Write to connection. - if err := coordinator.WriteTLV(w, byte(ResponseKillCopyShard), buf); err != nil { - s.Logger.Error("kill copy shard WriteTLV fail", zap.Error(err)) - } + setError(&resp.CommonResp, e) + s.writeResponse(w, ResponseKillCopyShard, &resp) } func (s *Service) handleRemoveShard(conn net.Conn) error { s.Logger.Info("handleRemoveShard") - buf, err := coordinator.ReadLV(conn) - if err != nil { - s.Logger.Error("unable to read length-value", zap.Error(err)) - return err - } - var req RemoveShardRequest - if err := json.Unmarshal(buf, &req); err != nil { - s.Logger.Error("unmarshal fail.", zap.Error(err)) + if err := s.readRequest(conn, &req); err != nil { return err } @@ -316,6 +333,10 @@ func (s *Service) handleRemoveShard(conn net.Conn) error { } if s.Node.ID == ni.ID { + shard := s.TSDBStore.Shard(req.ShardID) + if shard == nil { + return errors.New("Shard not found") + } if err := s.TSDBStore.DeleteShard(req.ShardID); err != nil { s.Logger.Error("DeleteShard fail.", zap.Error(err)) return err @@ -332,36 +353,13 @@ func (s *Service) handleRemoveShard(conn net.Conn) error { func (s *Service) removeShardResponse(w io.Writer, e error) { // Build response. var resp RemoveShardResponse - if e != nil { - resp.Code = 1 - resp.Msg = e.Error() - } else { - resp.Code = 0 - resp.Msg = "ok" - } - - // Marshal response to binary. - buf, err := json.Marshal(&resp) - if err != nil { - s.Logger.Error("error marshalling remove shard response", zap.Error(err)) - return - } - - // Write to connection. - if err := coordinator.WriteTLV(w, byte(ResponseRemoveShard), buf); err != nil { - s.Logger.Error("remove shard WriteTLV fail", zap.Error(err)) - } + setError(&resp.CommonResp, e) + s.writeResponse(w, ResponseRemoveShard, &resp) } func (s *Service) handleRemoveDataNode(conn net.Conn) error { - buf, err := coordinator.ReadLV(conn) - if err != nil { - s.Logger.Error("unable to read length-value", zap.Error(err)) - return err - } - var req RemoveDataNodeRequest - if err := json.Unmarshal(buf, &req); err != nil { + if err := s.readRequest(conn, &req); err != nil { return err } @@ -378,25 +376,166 @@ func (s *Service) handleRemoveDataNode(conn net.Conn) error { func (s *Service) removeDataNodeResponse(w io.Writer, e error) { // Build response. var resp RemoveDataNodeResponse - if e != nil { - resp.Code = 1 - resp.Msg = e.Error() - } else { - resp.Code = 0 - resp.Msg = "ok" + setError(&resp.CommonResp, e) + s.writeResponse(w, ResponseRemoveDataNode, &resp) +} + +func (s *Service) handleFreezeDataNode(conn net.Conn) error { + var req FreezeDataNodeRequest + if err := s.readRequest(conn, &req); err != nil { + return err } - // Marshal response to binary. - buf, err := json.Marshal(&resp) + ni, err := s.MetaClient.DataNodeByTCPHost(req.DataNodeAddr) if err != nil { - s.Logger.Error("error marshalling remove data node response", zap.Error(err)) - return + return err + } else if ni == nil { + return fmt.Errorf("not find data node by addr:%s", req.DataNodeAddr) } - // Write to connection. - if err := coordinator.WriteTLV(w, byte(ResponseRemoveDataNode), buf); err != nil { - s.Logger.Error("remove data node WriteTLV fail", zap.Error(err)) + if req.Freeze { + return s.MetaClient.FreezeDataNode(ni.ID) + } else { + return s.MetaClient.UnfreezeDataNode(ni.ID) + } +} + +func (s *Service) freezeDataNodeResponse(w io.Writer, e error) { + // Build response. + var resp FreezeDataNodeResponse + setError(&resp.CommonResp, e) + s.writeResponse(w, ResponseFreezeDataNode, &resp) +} + +func (s *Service) handleNodeShards(conn net.Conn) ([]uint64, error) { + var req GetNodeShardsRequest + if err := s.readRequest(conn, &req); err != nil { + return nil, err + } + if req.NodeID < 1 { + return nil, errors.New("Node ID should not be empty") + } + + shards := make([]uint64, 0) + dbs := s.MetaClient.Databases() + for _, db := range dbs { + infoList := db.ShardInfos() + for _, info := range infoList { + if info.OwnedBy(req.NodeID) { + shards = append(shards, info.ID) + } + } + } + sort.Slice(shards, func(i, j int) bool { + return shards[i] < shards[j] + }) + + return shards, nil +} + +func (s *Service) handleShards(conn net.Conn) (*meta.RetentionPolicyInfo, error) { + var req GetShardsRequest + if err := s.readRequest(conn, &req); err != nil { + return nil, err + } + if req.Database == "" || req.RetentionPolicy == "" { + return nil, errors.New("Both database and retention policy should be specified") + } + return s.MetaClient.RetentionPolicy(req.Database, req.RetentionPolicy) +} + +func fromMetaOwners(owners []meta.ShardOwner) []uint64 { + nodes := make([]uint64, len(owners)) + for i, owner := range owners { + nodes[i] = owner.NodeID + } + return nodes +} + +func fromMetaShards(shards []meta.ShardInfo) []Shard { + list := make([]Shard, len(shards)) + for i, shard := range shards { + list[i].ID = shard.ID + list[i].Nodes = fromMetaOwners(shard.Owners) + } + return list +} + +func (s *Service) shardsResponse(w io.Writer, rp *meta.RetentionPolicyInfo, e error) { + // Build response. + var resp ShardsResponse + setError(&resp.CommonResp, e) + if rp != nil { + resp.Rp = rp.Name + resp.Duration = int64(rp.Duration / time.Millisecond) + resp.GroupDuration = int64(rp.ShardGroupDuration / time.Millisecond) + resp.Replica = rp.ReplicaN + resp.Groups = make([]ShardGroup, len(rp.ShardGroups)) + for i, g := range rp.ShardGroups { + resp.Groups[i].ID = g.ID + resp.Groups[i].StartTime = g.StartTime.UnixNano() / MILLISECOND + resp.Groups[i].EndTime = g.EndTime.UnixNano() / MILLISECOND + resp.Groups[i].DeletedAt = g.DeletedAt.UnixNano() / MILLISECOND + resp.Groups[i].TruncatedAt = g.TruncatedAt.UnixNano() / MILLISECOND + resp.Groups[i].Shards = fromMetaShards(g.Shards) + } + } + + s.writeResponse(w, ResponseShards, &resp) +} + +func (s *Service) nodeShardsResponse(w io.Writer, shards []uint64, e error) { + // Build response. + var resp NodeShardsResponse + setError(&resp.CommonResp, e) + resp.Shards = shards + + s.writeResponse(w, ResponseNodeShards, &resp) +} + +func (s *Service) handleShard(conn net.Conn) (string, string, *meta.ShardInfo, *meta.ShardGroupInfo, error) { + var ( + req GetShardRequest + err error + db, rp string + groupInfo *meta.ShardGroupInfo + ) + if err = s.readRequest(conn, &req); err != nil { + goto NOT_FOUND + } + if req.ShardID < 1 { + err = errors.New("ShardID should be specified") + goto NOT_FOUND + } + db, rp, groupInfo = s.MetaClient.ShardOwner(req.ShardID) + fmt.Println("shardId:", req.ShardID, "=>", db, rp, groupInfo) + if db == "" || rp == "" || groupInfo == nil { + err = errors.New("Specified shard could not be found") + goto NOT_FOUND + } + for _, shard := range groupInfo.Shards { + if shard.ID == req.ShardID { + return db, rp, &shard, groupInfo, nil + } + } +NOT_FOUND: + return "", "", nil, nil, err +} + +func (s *Service) shardResponse(w io.Writer, db, rp string, shard *meta.ShardInfo, groupInfo *meta.ShardGroupInfo, e error) { + var resp ShardResponse + setError(&resp.CommonResp, e) + if e == nil { + resp.ID = shard.ID + resp.DB = db + resp.Rp = rp + resp.Nodes = fromMetaOwners(shard.Owners) + resp.GroupID = groupInfo.ID + resp.Begin = groupInfo.StartTime.UnixNano() / MILLISECOND + resp.End = groupInfo.EndTime.UnixNano() / MILLISECOND + resp.Truncated = groupInfo.TruncatedAt.UnixNano() / MILLISECOND } + s.writeResponse(w, ResponseShard, &resp) } func (s *Service) handleShowDataNodes() ([]DataNode, error) { @@ -407,7 +546,7 @@ func (s *Service) handleShowDataNodes() ([]DataNode, error) { var dataNodes []DataNode for _, n := range nodes { - dataNodes = append(dataNodes, DataNode{ID: n.ID, TcpAddr: n.TCPHost, HttpAddr: n.Host}) + dataNodes = append(dataNodes, DataNode{ID: n.ID, TcpAddr: n.TCPHost, HttpAddr: n.Host, Freezed: s.MetaClient.IsDataNodeFreezed(n.ID)}) } return dataNodes, nil } @@ -415,26 +554,19 @@ func (s *Service) handleShowDataNodes() ([]DataNode, error) { func (s *Service) showDataNodesResponse(w io.Writer, nodes []DataNode, e error) { // Build response. var resp ShowDataNodesResponse - if e != nil { + setError(&resp.CommonResp, e) + resp.DataNodes = nodes + s.writeResponse(w, ResponseShowDataNodes, &resp) +} + +func setError(resp *CommonResp, err error) { + if err != nil { resp.Code = 1 - resp.Msg = e.Error() + resp.Msg = err.Error() } else { resp.Code = 0 resp.Msg = "ok" } - resp.DataNodes = nodes - - // Marshal response to binary. - buf, err := json.Marshal(&resp) - if err != nil { - s.Logger.Error("error marshalling show data nodes response", zap.Error(err)) - return - } - - // Write to connection. - if err := coordinator.WriteTLV(w, byte(ResponseShowDataNodes), buf); err != nil { - s.Logger.Error("show data nodes WriteTLV fail", zap.Error(err)) - } } type CommonResp struct { @@ -451,7 +583,7 @@ type TruncateShardResponse struct { type CopyShardRequest struct { SourceNodeAddr string `json:"source_node_address"` - DestNodeAddr string `json:"dest_node_address"` + DestNodeAddr string `json:"dest_node_address"` // is this necessary? ShardID uint64 `json:"shard_id"` } @@ -463,7 +595,7 @@ type CopyShardTask struct { Database string `json:"database"` Rp string `json:"retention_policy"` ShardID uint64 `json:"shard_id"` - TotalSize uint64 `json:"total_size"` + TotalSize uint64 `json:"total_size"` // is this necessary? currently it's ignored. CurrentSize uint64 `json:"current_size"` Source string `json:"source"` Destination string `json:"destination"` @@ -484,6 +616,18 @@ type KillCopyShardResponse struct { CommonResp } +type GetShardsRequest struct { + Database, RetentionPolicy string +} + +type GetNodeShardsRequest struct { + NodeID uint64 +} + +type GetShardRequest struct { + ShardID uint64 +} + type RemoveShardRequest struct { DataNodeAddr string `json:"data_node_addr"` ShardID uint64 `json:"shard_id"` @@ -501,10 +645,59 @@ type RemoveDataNodeResponse struct { CommonResp } +type FreezeDataNodeRequest struct { + DataNodeAddr string `json:"data_node_addr"` + Freeze bool `json:"freeze"` +} +type FreezeDataNodeResponse struct { + CommonResp +} + type DataNode struct { ID uint64 `json:"id"` TcpAddr string `json:"tcp_addr"` HttpAddr string `json:"http_addr"` + Freezed bool `json:"freezed"` +} + +type ShardGroup struct { + ID uint64 `json:"id"` + StartTime int64 `json:"begin"` + EndTime int64 `json:"end"` + DeletedAt int64 `json:"deleted_at"` + TruncatedAt int64 `json:"truncated_at"` + Shards []Shard `json:"shards"` +} + +type Shard struct { + ID uint64 `json:"id"` + Nodes []uint64 `json:"nodes"` +} + +type ShardsResponse struct { + CommonResp + Rp string `json:"rp"` + Replica int `json:"replica"` + Duration int64 `json:"duration"` + GroupDuration int64 `json:"group_duration"` + Groups []ShardGroup `json:"groups"` +} + +type NodeShardsResponse struct { + CommonResp + Shards []uint64 `json:"shards"` +} + +type ShardResponse struct { + CommonResp + ID uint64 `json:"id"` + DB string `json:"db"` + Rp string `json:"rp"` + Nodes []uint64 `json:"nodes"` + GroupID uint64 `json:"groupId"` + Begin int64 `json:"begin"` + End int64 `json:"end"` + Truncated int64 `json:"truncated"` } type ShowDataNodesResponse struct { @@ -517,23 +710,33 @@ type RequestType byte const ( // RequestTruncateShard represents a request for truncating shard. - RequestTruncateShard RequestType = 1 - RequestCopyShard = 2 - RequestCopyShardStatus = 3 - RequestKillCopyShard = 4 - RequestRemoveShard = 5 - RequestRemoveDataNode = 6 - RequestShowDataNodes = 7 + _ RequestType = iota + RequestTruncateShard + RequestCopyShard + RequestCopyShardStatus + RequestKillCopyShard + RequestRemoveShard + RequestRemoveDataNode + RequestShowDataNodes + RequestShards + RequestShard + RequestFreezeDataNode + RequestNodeShards ) type ResponseType byte const ( - ResponseTruncateShard ResponseType = 1 - ResponseCopyShard = 2 - ResponseCopyShardStatus = 3 - ResponseKillCopyShard = 4 - ResponseRemoveShard = 5 - ResponseRemoveDataNode = 6 - ResponseShowDataNodes = 7 + _ ResponseType = iota + ResponseTruncateShard + ResponseCopyShard + ResponseCopyShardStatus + ResponseKillCopyShard + ResponseRemoveShard + ResponseRemoveDataNode + ResponseShowDataNodes + ResponseShards + ResponseShard + ResponseFreezeDataNode + ResponseNodeShards ) diff --git a/services/controller/shard.go b/services/controller/shard.go index 3a9f316..2cd4004 100644 --- a/services/controller/shard.go +++ b/services/controller/shard.go @@ -1,307 +1,65 @@ package controller import ( - "encoding/json" - "errors" "fmt" - "io" - "math" "os" "path/filepath" "strconv" - "sync" - "time" - "github.com/influxdata/influxdb" - "github.com/influxdata/influxdb/cmd/influxd/backup_util" - tarstream "github.com/influxdata/influxdb/pkg/tar" - "github.com/influxdata/influxdb/services/meta" - "github.com/influxdata/influxdb/services/snapshotter" - "github.com/influxdata/influxdb/tcp" - "github.com/influxdata/influxdb/tsdb" + "github.com/angopher/chronus/services/migrate" + "github.com/angopher/chronus/x" "go.uber.org/zap" ) -type ShardCopyTask struct { - start time.Time - db string - rp string - totalSize uint64 - currentSize uint64 - shardId uint64 - source string - destination string - closer interface { - Close() error - } -} - -type ShardCopyManager struct { - mutex sync.Mutex - tasks map[uint64]*ShardCopyTask - maxRunning int -} - -func NewCopyManager(max int) *ShardCopyManager { - return &ShardCopyManager{ - maxRunning: max, - tasks: make(map[uint64]*ShardCopyTask), - } -} - -func (c *ShardCopyManager) Add(task *ShardCopyTask) error { - c.mutex.Lock() - defer c.mutex.Unlock() - _, ok := c.tasks[task.shardId] - if ok { - return errors.New("shard task already exists") - } - c.tasks[task.shardId] = task - return nil -} - -func (c *ShardCopyManager) Remove(id uint64) { - c.mutex.Lock() - defer c.mutex.Unlock() - delete(c.tasks, id) - return -} - -func (c *ShardCopyManager) Tasks() []*ShardCopyTask { - c.mutex.Lock() - defer c.mutex.Unlock() - var tasks []*ShardCopyTask - for _, task := range c.tasks { - tasks = append(tasks, task) - } - return tasks -} - -func (c *ShardCopyManager) Kill(id uint64, source, dest string) { - c.mutex.Lock() - defer c.mutex.Unlock() - t, ok := c.tasks[id] - if !ok { - return - } - if source == t.source && dest == t.destination { - t.closer.Close() - delete(c.tasks, id) - } -} - -type ShardCopier struct { - Node *influxdb.Node - Logger *zap.Logger - Manager interface { - Add(task *ShardCopyTask) error - Remove(id uint64) - Tasks() []*ShardCopyTask - Kill(id uint64, source, dest string) - } - - MetaClient interface { - ShardOwner(shardID uint64) (database, policy string, sgi *meta.ShardGroupInfo) - AddShardOwner(shardID, nodeID uint64) error - } - TSDBStore interface { - Path() string - ShardRelativePath(id uint64) (string, error) - CreateShard(database, retentionPolicy string, shardID uint64, enabled bool) error - Shard(id uint64) *tsdb.Shard - } -} - -func (s *ShardCopier) WithLogger(log *zap.Logger) { - s.Logger = log.With(zap.String("service", "Controller.ShardCopier")) -} - -func fileExists(fileName string) bool { - _, err := os.Stat(fileName) - return err == nil -} - -func (s *ShardCopier) Query() []CopyShardTask { - var ts []CopyShardTask - tasks := s.Manager.Tasks() - for _, t := range tasks { - task := CopyShardTask{ - Database: t.db, - Rp: t.rp, - ShardID: t.shardId, - TotalSize: t.totalSize, - CurrentSize: t.currentSize, - Source: t.source, - Destination: t.destination, - } - ts = append(ts, task) - } - return ts -} - -func (s *ShardCopier) Kill(shardId uint64, source, destination string) { - s.Manager.Kill(shardId, source, destination) -} - -func (s *ShardCopier) CopyShard(sourceAddr string, shardId uint64) error { - // 1.检查本地是否已经存在shard - // 2.检查是否可以进行此shard的copy任务: 任务管理器 - // 3.检查本地是否有残留的脏数据, 并清理掉 - // 4.下载shard备份 - // 5.解压shard数据包 - // 6.创建shard +func (s *Service) copyShard(sourceAddr string, shardId uint64) error { + task := migrate.Task{} + task.SrcHost = sourceAddr + task.ShardId = shardId db, rp, sgi := s.MetaClient.ShardOwner(shardId) if sgi == nil { return fmt.Errorf("shard %d not exists", shardId) } + task.Database = db + task.Retention = rp path := filepath.Join(s.TSDBStore.Path(), db, rp, strconv.FormatUint(shardId, 10)) - if fileExists(path) { - return fmt.Errorf("path:[%s] exists", path) + if x.Exists(path) != x.NotExisted { + return fmt.Errorf("local shard:[%s] exists", path) } + task.DstStorePath = path - copyDir := s.TSDBStore.Path() + "/.copy_shard" + copyDir := filepath.Join(s.TSDBStore.Path(), ".copy_shard") os.MkdirAll(copyDir, 0755) - tmpPath := fmt.Sprintf("%s/shard_%d", copyDir, shardId) - defer os.RemoveAll(tmpPath) - - if !fileExists(tmpPath) { - req := &snapshotter.Request{ - Type: snapshotter.RequestShardBackup, - BackupDatabase: db, - BackupRetentionPolicy: rp, - ShardID: shardId, - //Since: cmd.since, - //ExportStart: cmd.start, - //ExportEnd: cmd.end, - } - fmt.Println("----------- tmpPath:", tmpPath) - if err := s.downloadAndVerify(req, sourceAddr, tmpPath, nil); err != nil { - return err - } - } - - fmt.Println("----------- shard path:", path) - if _, err := os.Stat(tmpPath); err == nil || os.IsNotExist(err) { - if err := s.unpackTar(tmpPath, path); err != nil { - return err - } - } + task.TmpStorePath = copyDir - sh := s.TSDBStore.Shard(shardId) - if sh == nil { - if err := s.TSDBStore.CreateShard(db, rp, shardId, true); err != nil { - return err - } - } - - return s.MetaClient.AddShardOwner(shardId, s.Node.ID) -} - -func (s *ShardCopier) downloadAndVerify(req *snapshotter.Request, host, path string, validator func(string) error) error { - tmppath := path + backup_util.Suffix - if err := s.download(req, host, tmppath); err != nil { - os.Remove(tmppath) + err := s.migrateManager.Add(&task) + if err != nil { return err } - if validator != nil { - if err := validator(tmppath); err != nil { - if rmErr := os.Remove(tmppath); rmErr != nil { - s.Logger.Sugar().Errorf("Error cleaning up temporary file: %v", rmErr) - } - return err - } - } + err = <-task.C - f, err := os.Stat(tmppath) if err != nil { return err } - // There was nothing downloaded, don't create an empty backup file. - if f.Size() == 0 { - return os.Remove(tmppath) - } - - // Rename temporary file to final path. - if err := os.Rename(tmppath, path); err != nil { - return fmt.Errorf("rename: %s", err) + sh := s.TSDBStore.Shard(shardId) + if sh != nil { + return fmt.Errorf("Shard %d is existed which is unexpected after backuping", shardId) } - return nil -} - -// unpackTar will restore a single tar archive to the data dir -func (s *ShardCopier) unpackTar(tarFile, shardPath string) error { - s.Logger.Sugar().Infof("Restoring from backup %s\n", shardPath) - f, err := os.Open(tarFile) + err = s.TSDBStore.CreateShard(task.Database, task.Retention, task.ShardId, true) if err != nil { + s.Logger.Warn("Failed to load shard into memory", zap.Error(err)) return err } - defer f.Close() - os.MkdirAll(shardPath, 0755) - - return tarstream.Restore(f, shardPath) -} - -func (s *ShardCopier) download(req *snapshotter.Request, host, path string) error { - // Create local file to write to. - f, err := os.Create(path) + err = s.MetaClient.AddShardOwner(task.ShardId, s.Node.ID) if err != nil { - return fmt.Errorf("open temp file: %s", err) - } - defer f.Close() - - min := 2 * time.Second - for i := 0; i < 2; i++ { - if err = func() error { - // Connect to snapshotter service. - conn, err := tcp.Dial("tcp", host, snapshotter.MuxHeader) - if err != nil { - return err - } - defer conn.Close() - - task := &ShardCopyTask{ - shardId: req.ShardID, - source: host, - closer: conn, - } - s.Manager.Add(task) - defer s.Manager.Remove(task.shardId) - - // Write the request type - _, err = conn.Write([]byte{byte(req.Type)}) - if err != nil { - return err - } - - // Write the request - if err := json.NewEncoder(conn).Encode(req); err != nil { - return fmt.Errorf("encode snapshot request: %s", err) - } - - // Read snapshot from the connection - //TODO: 1.Rate limit? - // 2.How to get progress? - if n, err := io.Copy(f, conn); err != nil || n == 0 { - return fmt.Errorf("copy backup to file: err=%v, n=%d", err, n) - } - return nil - }(); err == nil { - break - } else if err != nil { - backoff := time.Duration(math.Pow(3.8, float64(i))) * time.Millisecond - if backoff < min { - backoff = min - } - s.Logger.Sugar().Errorf("Download shard %v failed %s. Waiting %v and retrying (%d)...\n", req.ShardID, err, backoff, i) - time.Sleep(backoff) - } + s.Logger.Warn("Failed to add as owner", zap.Error(err)) + return err } - + s.Logger.Info("Successfully add as owner", zap.Uint64("shard", task.ShardId)) return err } diff --git a/services/hh/config.go b/services/hh/config.go index 0e1fe52..dcc292c 100644 --- a/services/hh/config.go +++ b/services/hh/config.go @@ -41,7 +41,7 @@ type Config struct { Dir string `toml:"dir"` MaxSize int64 `toml:"max-size"` MaxAge toml.Duration `toml:"max-age"` - RetryConcurrency int64 `toml:"retry-concurrency"` + RetryConcurrency int32 `toml:"retry-concurrency"` RetryRateLimit int64 `toml:"retry-rate-limit"` RetryInterval toml.Duration `toml:"retry-interval"` RetryMaxInterval toml.Duration `toml:"retry-max-interval"` diff --git a/services/hh/config_test.go b/services/hh/config_test.go index 2f57a57..25d2c71 100644 --- a/services/hh/config_test.go +++ b/services/hh/config_test.go @@ -1,16 +1,15 @@ -package hh_test +package hh import ( "testing" "time" "github.com/BurntSushi/toml" - "github.com/influxdata/influxdb/services/hh" ) func TestConfigParse(t *testing.T) { // Parse configuration. - var c hh.Config + var c Config if _, err := toml.Decode(` enabled = false retry-interval = "10m" @@ -56,7 +55,7 @@ purge-interval = "1h" func TestDefaultDisabled(t *testing.T) { // Parse empty configuration. - var c hh.Config + var c Config if _, err := toml.Decode(``, &c); err != nil { t.Fatal(err) } @@ -66,7 +65,7 @@ func TestDefaultDisabled(t *testing.T) { } // Default configuration. - c = hh.NewConfig() + c = NewConfig() if exp := false; c.Enabled == true { t.Fatalf("unexpected default enabled value: got %v, exp %v", c.Enabled, exp) } diff --git a/services/hh/limiter.go b/services/hh/limiter.go deleted file mode 100644 index b2a69f9..0000000 --- a/services/hh/limiter.go +++ /dev/null @@ -1,61 +0,0 @@ -package hh - -import "time" - -type limiter struct { - count int64 - limit int64 - start time.Time - delay float64 -} - -// NewRateLimiter returns a new limiter configured to restrict a process to the limit per second. -// limit is the maximum amount that can be used per second. The limit should be > 0. A limit -// <= 0, will not limit the processes. -func NewRateLimiter(limit int64) *limiter { - return &limiter{ - start: time.Now(), - limit: limit, - delay: 0.5, - } -} - -// Update updates the amount used -func (t *limiter) Update(count int) { - t.count += int64(count) -} - -// Delay returns the amount of time, up to 1 second, that caller should wait -// to maintain the configured rate -func (t *limiter) Delay() time.Duration { - if t.limit > 0 { - - delta := time.Now().Sub(t.start).Seconds() - rate := int64(float64(t.count) / delta) - - // Determine how far off from the max rate we are - delayAdj := float64((t.limit - rate)) / float64(t.limit) - - // Don't adjust by more than 1 second at a time - delayAdj = t.clamp(delayAdj, -1, 1) - - t.delay -= delayAdj - if t.delay < 0 { - t.delay = 0 - } - - return time.Duration(t.delay) * time.Second - } - return time.Duration(0) -} - -func (t *limiter) clamp(value, min, max float64) float64 { - if value < min { - return min - } - - if value > max { - return max - } - return value -} diff --git a/services/hh/limiter_test.go b/services/hh/limiter_test.go deleted file mode 100644 index 5edb6e0..0000000 --- a/services/hh/limiter_test.go +++ /dev/null @@ -1,47 +0,0 @@ -package hh - -import ( - "testing" - "time" -) - -func TestLimiter(t *testing.T) { - l := NewRateLimiter(0) - l.Update(500) - if l.Delay().Nanoseconds() != 0 { - t.Errorf("limiter with no limit mismatch: got %v, exp 0", l.Delay()) - } -} - -func TestLimiterWithinLimit(t *testing.T) { - if testing.Short() { - t.Skip("Shipping TestLimiterWithinLimit") - } - - l := NewRateLimiter(1000) - for i := 0; i < 100; i++ { - // 50 ever 100ms = 500/s which should be within the rate - l.Update(50) - l.Delay() - time.Sleep(100 * time.Millisecond) - } - - // Should not have any delay - delay := l.Delay().Seconds() - if exp := int(0); int(delay) != exp { - t.Errorf("limiter rate mismatch: got %v, exp %v", int(delay), exp) - } - -} - -func TestLimiterExceeded(t *testing.T) { - l := NewRateLimiter(1000) - for i := 0; i < 10; i++ { - l.Update(200) - l.Delay() - } - delay := l.Delay().Seconds() - if int(delay) == 0 { - t.Errorf("limiter rate mismatch. expected non-zero delay") - } -} diff --git a/services/hh/node_processor.go b/services/hh/node_processor.go index e00f320..9dfffb2 100644 --- a/services/hh/node_processor.go +++ b/services/hh/node_processor.go @@ -1,17 +1,21 @@ package hh import ( + "context" "encoding/binary" "fmt" "io" - "log" "os" "sync" "time" - "github.com/influxdata/influxdb/models" "strings" "sync/atomic" + + "github.com/angopher/chronus/services/meta" + "github.com/influxdata/influxdb/models" + "go.uber.org/zap" + "golang.org/x/time/rate" ) const ( @@ -20,6 +24,12 @@ const ( writeNodeReqPoints = "writeNodeReqPoints" ) +var ( + // for concurrency control + maxActiveProcessorCount = int32(0) + activeProcessorCount = int32(0) +) + // NodeProcessor encapsulates a queue of hinted-handoff data for a node, and the // transmission of the data to the node. type NodeProcessor struct { @@ -28,7 +38,7 @@ type NodeProcessor struct { RetryMaxInterval time.Duration // Max interval between periodic write-to-node attempts. MaxSize int64 // Maximum size an underlying queue can get. MaxAge time.Duration // Maximum age queue data can get before purging. - RetryRateLimit int64 // Limits the rate data is sent to node. + RetryRateLimit int // Limits the rate data is sent to node. nodeID uint64 dir string @@ -41,7 +51,7 @@ type NodeProcessor struct { writer shardWriter stats *NodeProcessorStatistics - Logger *log.Logger + Logger *zap.SugaredLogger } type NodeProcessorStatistics struct { @@ -52,6 +62,10 @@ type NodeProcessorStatistics struct { WriteNodeReqPoints int64 } +func SetMaxActiveProcessorCount(n int32) { + maxActiveProcessorCount = n +} + // NewNodeProcessor returns a new NodeProcessor for the given node, using dir for // the hinted-handoff data. func NewNodeProcessor(nodeID uint64, dir string, w shardWriter, m metaClient) *NodeProcessor { @@ -66,10 +80,14 @@ func NewNodeProcessor(nodeID uint64, dir string, w shardWriter, m metaClient) *N writer: w, meta: m, stats: &NodeProcessorStatistics{}, - Logger: log.New(os.Stderr, "[handoff] ", log.LstdFlags), + Logger: zap.NewNop().Sugar(), } } +func (n *NodeProcessor) WithLogger(logger *zap.Logger) { + n.Logger = logger.With(zap.String("service", "hh_processor")).Sugar() +} + // Open opens the NodeProcessor. It will read and write data present in dir, and // start transmitting data to the node. A NodeProcessor must be opened before it // can accept hinted data. @@ -186,49 +204,99 @@ func (n *NodeProcessor) LastModified() (time.Time, error) { func (n *NodeProcessor) run() { defer n.wg.Done() - currInterval := time.Duration(n.RetryInterval) - if currInterval > time.Duration(n.RetryMaxInterval) { - currInterval = time.Duration(n.RetryMaxInterval) + waitTime := time.Duration(n.RetryInterval) + if waitTime > time.Duration(n.RetryMaxInterval) { + waitTime = time.Duration(n.RetryMaxInterval) } + purgeTimer := time.NewTimer(n.PurgeInterval) + defer purgeTimer.Stop() + sendingTimer := time.NewTimer(waitTime) + defer sendingTimer.Stop() for { select { case <-n.done: return - case <-time.After(n.PurgeInterval): + case <-purgeTimer.C: if err := n.queue.PurgeOlderThan(time.Now().Add(-n.MaxAge)); err != nil { - n.Logger.Printf("failed to purge for node %d: %s", n.nodeID, err.Error()) + n.Logger.Warnf("failed to purge for node %d: %s", n.nodeID, err.Error()) + } + purgeTimer.Reset(n.PurgeInterval) + + case <-sendingTimer.C: + waitTime = n.sendingLoop(waitTime) + sendingTimer.Reset(waitTime) + + } + } +} + +func concurrencyAllow() bool { + if maxActiveProcessorCount < 1 { + return true + } + waiter := time.NewTimer(time.Second) + defer waiter.Stop() + for { + select { + case <-waiter.C: + // timeout + return false + default: + if atomic.AddInt32(&activeProcessorCount, 1) <= maxActiveProcessorCount { + return true } + // restore & next + atomic.AddInt32(&activeProcessorCount, -1) + } + } +} - case <-time.After(currInterval): - limiter := NewRateLimiter(n.RetryRateLimit) - for { - c, err := n.SendWrite() - if err != nil { - if err == io.EOF { - // No more data, return to configured interval - currInterval = time.Duration(n.RetryInterval) - } else { - currInterval = currInterval * 2 - if currInterval > time.Duration(n.RetryMaxInterval) { - currInterval = time.Duration(n.RetryMaxInterval) - } - } - break - } - - // Success! Ensure backoff is cancelled. - currInterval = time.Duration(n.RetryInterval) - - // Update how many bytes we've sent - limiter.Update(c) - - // Block to maintain the throughput rate - time.Sleep(limiter.Delay()) +func (n *NodeProcessor) sendingLoop(curDelay time.Duration) (nextDelay time.Duration) { + var ( + sent int + err error + ) + + // concurrency check + if maxActiveProcessorCount > 0 { + if !concurrencyAllow() { + n.Logger.Info("concurrency control, skip scheduling once") + return n.RetryInterval + } + defer atomic.AddInt32(&activeProcessorCount, -1) + } + + // Bytes rate limit + if n.RetryRateLimit > 0 { + bytesLimiter := rate.NewLimiter(rate.Limit(n.RetryRateLimit), 10*n.RetryRateLimit) + defer func() { + if sent > 0 { + n.Logger.Infof("write to %d with %d bytes", n.nodeID, sent) + bytesLimiter.WaitN(context.Background(), sent) } + }() + } + + sent, err = n.SendWrite() + if err == nil { + // Success! Ensure backoff is cancelled. + nextDelay = n.RetryInterval + return + } + + if err == io.EOF { + // No more data, return to configured interval + nextDelay = n.RetryInterval + } else { + // backoff + nextDelay = 2 * curDelay + if nextDelay > n.RetryMaxInterval { + nextDelay = n.RetryMaxInterval } } + return } // SendWrite attempts to sent the current block of hinted data to the target node. If successful, @@ -255,10 +323,10 @@ func (n *NodeProcessor) SendWrite() (int, error) { // unmarshal the byte slice back to shard ID and points shardID, points, err := unmarshalWrite(buf) if err != nil { - n.Logger.Printf("unmarshal write failed: %v", err) + n.Logger.Warnf("unmarshal write failed: %v", err) // Try to skip it. if err := n.queue.Advance(); err != nil { - n.Logger.Printf("failed to advance queue for node %d: %s", n.nodeID, err.Error()) + n.Logger.Warnf("failed to advance queue for node %d: %s", n.nodeID, err.Error()) } return 0, err } @@ -271,7 +339,7 @@ func (n *NodeProcessor) SendWrite() (int, error) { atomic.AddInt64(&n.stats.WriteNodeReqPoints, int64(len(points))) if err := n.queue.Advance(); err != nil { - n.Logger.Printf("failed to advance queue for node %d: %s", n.nodeID, err.Error()) + n.Logger.Warnf("failed to advance queue for node %d: %s", n.nodeID, err.Error()) } return len(buf), nil @@ -298,8 +366,8 @@ func (n *NodeProcessor) Tail() string { // Active returns whether this node processor is for a currently active node. func (n *NodeProcessor) Active() (bool, error) { nio, err := n.meta.DataNode(n.nodeID) - if err != nil { - n.Logger.Printf("failed to determine if node %d is active: %s", n.nodeID, err.Error()) + if err != nil && err != meta.ErrNodeNotFound { + n.Logger.Warnf("failed to determine if node %d is active: %s", n.nodeID, err.Error()) return false, err } return nio != nil, nil diff --git a/services/hh/node_processor_test.go b/services/hh/node_processor_test.go index 10dd560..c8b40c5 100644 --- a/services/hh/node_processor_test.go +++ b/services/hh/node_processor_test.go @@ -8,8 +8,7 @@ import ( "time" "github.com/influxdata/influxdb/models" - - "github.com/angopher/chronus/services/meta" + "github.com/influxdata/influxdb/services/meta" ) type fakeShardWriter struct { @@ -36,7 +35,8 @@ func TestNodeProcessorSendBlock(t *testing.T) { // expected data to be queue and sent to the shardWriter var expShardID, expNodeID, count = uint64(100), uint64(200), 0 - pt := models.MustNewPoint("cpu", models.Tags{"foo": "bar"}, models.Fields{"value": 1.0}, time.Unix(0, 0)) + tag := models.Tag{Key: []byte("foo"), Value: []byte("bar")} + pt := models.MustNewPoint("cpu", models.Tags{tag}, models.Fields{"value": 1.0}, time.Unix(0, 0)) sh := &fakeShardWriter{ ShardWriteFn: func(shardID, nodeID uint64, points []models.Point) error { diff --git a/services/hh/queue.go b/services/hh/queue.go index a21c948..0fbf199 100644 --- a/services/hh/queue.go +++ b/services/hh/queue.go @@ -621,6 +621,10 @@ func (l *segment) lastModified() (time.Time, error) { l.mu.RLock() defer l.mu.RUnlock() + if l.file == nil { + return time.Time{}, os.ErrNotExist + } + stats, err := os.Stat(l.file.Name()) if err != nil { return time.Time{}, err diff --git a/services/hh/service.go b/services/hh/service.go index 211df93..de6b124 100644 --- a/services/hh/service.go +++ b/services/hh/service.go @@ -3,17 +3,18 @@ package hh // import "github.com/influxdata/influxdb/services/hh" import ( "fmt" "io/ioutil" - "log" "os" "path/filepath" "strconv" "sync" "time" + "sync/atomic" + "github.com/influxdata/influxdb/models" - "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxdb/monitor/diagnostics" - "sync/atomic" + "github.com/influxdata/influxdb/services/meta" + "go.uber.org/zap" ) // ErrHintedHandoffDisabled is returned when attempting to use a @@ -33,7 +34,7 @@ type Service struct { processors map[uint64]*NodeProcessor - Logger *log.Logger + Logger *zap.SugaredLogger cfg Config shardWriter shardWriter @@ -59,18 +60,32 @@ type metaClient interface { func NewService(c Config, w shardWriter, m metaClient) *Service { //key := strings.Join([]string{"hh", c.Dir}, ":") //tags := map[string]string{"path": c.Dir} + SetMaxActiveProcessorCount(c.RetryConcurrency) return &Service{ cfg: c, closing: make(chan struct{}), processors: make(map[uint64]*NodeProcessor), - Logger: log.New(os.Stderr, "[handoff] ", log.LstdFlags), + Logger: zap.NewNop().Sugar(), shardWriter: w, MetaClient: m, stats: &HHStatistics{}, } } +func (s *Service) WithLogger(log *zap.Logger) { + s.Logger = log.With(zap.String("service", "handoff")).Sugar() +} + +func (s *Service) createProcessor(nodeID uint64) *NodeProcessor { + n := NewNodeProcessor(nodeID, s.pathforNode(nodeID), s.shardWriter, s.MetaClient) + n.RetryInterval = time.Duration(s.cfg.RetryInterval) + n.RetryMaxInterval = time.Duration(s.cfg.RetryMaxInterval) + n.RetryRateLimit = int(s.cfg.RetryRateLimit) + n.WithLogger(s.Logger.Desugar()) + return n +} + // Open opens the hinted handoff service. func (s *Service) Open() error { s.mu.Lock() @@ -79,7 +94,7 @@ func (s *Service) Open() error { // Allow Open to proceed, but don't do anything. return nil } - s.Logger.Printf("Starting hinted handoff service") + s.Logger.Info("Starting hinted handoff service") s.closing = make(chan struct{}) // Register diagnostics if a Monitor service is available. @@ -88,7 +103,7 @@ func (s *Service) Open() error { } // Create the root directory if it doesn't already exist. - s.Logger.Printf("Using data dir: %v", s.cfg.Dir) + s.Logger.Infof("Using data dir: %v", s.cfg.Dir) if err := os.MkdirAll(s.cfg.Dir, 0700); err != nil { return fmt.Errorf("mkdir all: %s", err) } @@ -106,7 +121,7 @@ func (s *Service) Open() error { continue } - n := NewNodeProcessor(nodeID, s.pathforNode(nodeID), s.shardWriter, s.MetaClient) + n := s.createProcessor(nodeID) if err := n.Open(); err != nil { return err } @@ -121,7 +136,7 @@ func (s *Service) Open() error { // Close closes the hinted handoff service. func (s *Service) Close() error { - s.Logger.Println("shutting down hh service") + s.Logger.Info("shutting down hh service") s.mu.Lock() defer s.mu.Unlock() @@ -166,11 +181,6 @@ func (s *Service) Statistics(tags map[string]string) []models.Statistic { return statistics } -// SetLogger sets the internal logger to the logger passed in. -func (s *Service) SetLogger(l *log.Logger) { - s.Logger = l -} - // WriteShard queues the points write for shardID to node ownerID to handoff queue func (s *Service) WriteShard(shardID, ownerID uint64, points []models.Point) error { if !s.cfg.Enabled { @@ -190,7 +200,7 @@ func (s *Service) WriteShard(shardID, ownerID uint64, points []models.Point) err processor, ok = s.processors[ownerID] if !ok { - processor = NewNodeProcessor(ownerID, s.pathforNode(ownerID), s.shardWriter, s.MetaClient) + processor = s.createProcessor(ownerID) if err := processor.Open(); err != nil { return err } @@ -253,13 +263,13 @@ func (s *Service) purgeInactiveProcessors() { for k, v := range s.processors { lm, err := v.LastModified() if err != nil { - s.Logger.Printf("failed to determine LastModified for processor %d: %s", k, err.Error()) + s.Logger.Warnf("failed to determine LastModified for processor %d: %s", k, err.Error()) continue } active, err := v.Active() if err != nil { - s.Logger.Printf("failed to determine if node %d is active: %s", k, err.Error()) + s.Logger.Warnf("failed to determine if node %d is active: %s", k, err.Error()) continue } if active { @@ -273,11 +283,11 @@ func (s *Service) purgeInactiveProcessors() { } if err := v.Close(); err != nil { - s.Logger.Printf("failed to close node processor %d: %s", k, err.Error()) + s.Logger.Warnf("failed to close node processor %d: %s", k, err.Error()) continue } if err := v.Purge(); err != nil { - s.Logger.Printf("failed to purge node processor %d: %s", k, err.Error()) + s.Logger.Warnf("failed to purge node processor %d: %s", k, err.Error()) continue } delete(s.processors, k) diff --git a/services/meta/config_test.go b/services/meta/config_test.go new file mode 100644 index 0000000..318c3a5 --- /dev/null +++ b/services/meta/config_test.go @@ -0,0 +1,20 @@ +package meta + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestConfig(t *testing.T) { + cfg := NewConfig() + assert.NotNil(t, cfg.Validate()) + cfg.Dir = "some_value" + assert.Nil(t, cfg.Validate()) + + diag, err := cfg.Diagnostics() + assert.Nil(t, err) + assert.Equal(t, 1, len(diag.Rows)) + assert.Equal(t, 1, len(diag.Rows[0])) + assert.Equal(t, "some_value", diag.Rows[0][0]) +} diff --git a/services/meta/data.go b/services/meta/data.go index 4958567..98e39bd 100644 --- a/services/meta/data.go +++ b/services/meta/data.go @@ -2,6 +2,7 @@ package meta import ( "encoding/json" + "errors" "fmt" "sort" "time" @@ -14,8 +15,9 @@ import ( // Data represents the top level collection of all metadata. type Data struct { meta.Data - MetaNodes []meta.NodeInfo - DataNodes []meta.NodeInfo + MetaNodes []meta.NodeInfo + DataNodes []meta.NodeInfo + FreezedDataNodes []uint64 // data nodes that can't create new shard on MaxNodeID uint64 } @@ -24,18 +26,20 @@ type Data struct { func (data *Data) DataNode(id uint64) *meta.NodeInfo { for i := range data.DataNodes { if data.DataNodes[i].ID == id { - return &data.DataNodes[i] + // prevent unexpected modification + n := data.DataNodes[i] + return &n } } return nil } -// CreateDataNode adds a node to the metadata. -func (data *Data) CreateDataNode(host, tcpHost string) error { +// CreateDataNode adds a node to the metadata, return the nodeId(0 when an error occurred) and error +func (data *Data) CreateDataNode(host, tcpHost string) (uint64, error) { // Ensure a node with the same host doesn't already exist. for _, n := range data.DataNodes { if n.TCPHost == tcpHost || n.Host == host { - return ErrNodeExists + return 0, ErrNodeExists } } @@ -63,6 +67,65 @@ func (data *Data) CreateDataNode(host, tcpHost string) error { }) sort.Sort(meta.NodeInfos(data.DataNodes)) + return existingID, nil +} + +func existInNodes(nodes []meta.NodeInfo, id uint64) *meta.NodeInfo { + for _, n := range nodes { + if n.ID == id { + return &n + } + } + return nil +} + +func getFreezed(freezed []uint64, id uint64) int { + for i, n := range freezed { + if n == id { + return i + } + } + return -1 +} + +func (data *Data) UnfreezeDataNode(id uint64) error { + if id == 0 { + return ErrNodeIDRequired + } + + if existInNodes(data.DataNodes, id) == nil { + return ErrNodeNotFound + } + + i := getFreezed(data.FreezedDataNodes, id) + if i == -1 { + return ErrNodeNotFreezed + } + + data.FreezedDataNodes = append(data.FreezedDataNodes[:i], data.FreezedDataNodes[i+1:]...) + + return nil +} + +func (data *Data) IsFreezeDataNode(id uint64) bool { + return getFreezed(data.FreezedDataNodes, id) > -1 +} + +func (data *Data) FreezeDataNode(id uint64) error { + if id == 0 { + return ErrNodeIDRequired + } + + if existInNodes(data.DataNodes, id) == nil { + return ErrNodeNotFound + } + + if getFreezed(data.FreezedDataNodes, id) > -1 { + return ErrNodeAlreadyFreezed + } + + data.FreezedDataNodes = append(data.FreezedDataNodes, id) + return nil } @@ -95,7 +158,7 @@ func (data *Data) DeleteDataNode(id uint64) error { for ri, rp := range d.RetentionPolicies { for sgi, sg := range rp.ShardGroups { var ( - nodeOwnerFreqs = make(map[int]int) + nodeOwnerFreqs = make(map[uint64]int) orphanedShards []meta.ShardInfo ) // Look through all shards in the shard group and @@ -111,7 +174,7 @@ func (data *Data) DeleteDataNode(id uint64) error { if owner.NodeID == id { nodeIdx = i } - nodeOwnerFreqs[int(owner.NodeID)]++ + nodeOwnerFreqs[owner.NodeID]++ } if nodeIdx > -1 { @@ -137,18 +200,19 @@ func (data *Data) DeleteDataNode(id uint64) error { // Reassign any orphaned shards. Delete the node we're // dropping from the list of potential new owners. - delete(nodeOwnerFreqs, int(id)) + delete(nodeOwnerFreqs, id) for _, orphan := range orphanedShards { - newOwnerID, err := newShardOwner(orphan, nodeOwnerFreqs) - if err != nil { - return err + newOwnerID := newShardOwner(nodeOwnerFreqs) + if newOwnerID == 0 { + return errors.New(fmt.Sprint("No node can be reassigned to ", orphan.ID)) } for si, s := range sg.Shards { if s.ID == orphan.ID { sg.Shards[si].Owners = append(sg.Shards[si].Owners, meta.ShardOwner{NodeID: newOwnerID}) data.Databases[di].RetentionPolicies[ri].ShardGroups[sgi].Shards = sg.Shards + nodeOwnerFreqs[newOwnerID]++ break } } @@ -157,10 +221,16 @@ func (data *Data) DeleteDataNode(id uint64) error { } } } + + // Delete from freezed nodes if necessarily + if i := getFreezed(data.FreezedDataNodes, id); i > -1 { + data.FreezedDataNodes = append(data.FreezedDataNodes[:i], data.FreezedDataNodes[i+1:]...) + } + return nil } -func (data *Data) CloneNodes(src []meta.NodeInfo) []meta.NodeInfo { +func cloneNodes(src []meta.NodeInfo) []meta.NodeInfo { if len(src) == 0 { return []meta.NodeInfo{} } @@ -176,44 +246,49 @@ func (data *Data) CloneNodes(src []meta.NodeInfo) []meta.NodeInfo { // that currently owns the fewest number of shards. If multiple nodes // own the same (fewest) number of shards, then one of those nodes // becomes the new shard owner. -func newShardOwner(s meta.ShardInfo, ownerFreqs map[int]int) (uint64, error) { - var ( - minId = -1 - minFreq int - ) - - for id, freq := range ownerFreqs { - if minId == -1 || freq < minFreq { - minId, minFreq = int(id), freq - } +// ATTENTION: This method should guarantee that the result is stable between +// different metad instances. +func newShardOwner(freqs map[uint64]int) uint64 { + if len(freqs) < 1 { + return 0 } - if minId < 0 { - return 0, fmt.Errorf("cannot reassign shard %d due to lack of data nodes", s.ID) + type item struct { + id uint64 + freq int + } + arr := make([]item, 0, len(freqs)) + for id, freq := range freqs { + arr = append(arr, item{id, freq}) } + sort.Slice(arr, func(i, j int) bool { + if arr[i].freq != arr[j].freq { + return arr[i].freq < arr[j].freq + } + return arr[i].id < arr[j].id + }) - // Update the shard owner frequencies and set the new owner on the - // shard. - ownerFreqs[minId]++ - return uint64(minId), nil + return arr[0].id } // MetaNode returns a node by id. func (data *Data) MetaNode(id uint64) *meta.NodeInfo { for i := range data.MetaNodes { if data.MetaNodes[i].ID == id { - return &data.MetaNodes[i] + // prevent unexpected modification + n := data.MetaNodes[i] + return &n } } return nil } // CreateMetaNode will add a new meta node to the metastore -func (data *Data) CreateMetaNode(httpAddr, tcpAddr string) error { +func (data *Data) CreateMetaNode(httpAddr, tcpAddr string) (uint64, error) { // Ensure a node with the same host doesn't already exist. for _, n := range data.MetaNodes { if n.Host == httpAddr { - return ErrNodeExists + return 0, ErrNodeExists } } @@ -242,7 +317,7 @@ func (data *Data) CreateMetaNode(httpAddr, tcpAddr string) error { }) sort.Sort(meta.NodeInfos(data.MetaNodes)) - return nil + return existingID, nil } // DeleteMetaNode will remove the meta node from the store @@ -274,17 +349,20 @@ func (data *Data) Clone() *Data { other.Databases = data.CloneDatabases() other.Users = data.CloneUsers() - other.DataNodes = data.CloneNodes(data.DataNodes) - other.MetaNodes = data.CloneNodes(data.MetaNodes) + other.DataNodes = cloneNodes(data.DataNodes) + other.MetaNodes = cloneNodes(data.MetaNodes) + other.FreezedDataNodes = make([]uint64, len(data.FreezedDataNodes)) + copy(other.FreezedDataNodes, data.FreezedDataNodes) return &other } type DataJson struct { - Data []byte - MetaNodes []meta.NodeInfo - DataNodes []meta.NodeInfo - MaxNodeID uint64 + Data []byte + MetaNodes []meta.NodeInfo + DataNodes []meta.NodeInfo + MaxNodeID uint64 + FreezedDataNodes []uint64 } func (data *Data) marshal() ([]byte, error) { @@ -292,6 +370,7 @@ func (data *Data) marshal() ([]byte, error) { js.MetaNodes = data.MetaNodes js.DataNodes = data.DataNodes js.MaxNodeID = data.MaxNodeID + js.FreezedDataNodes = data.FreezedDataNodes var err error js.Data, err = data.Data.MarshalBinary() if err != nil { @@ -312,6 +391,7 @@ func (data *Data) unmarshal(buf []byte) error { data.MetaNodes = js.MetaNodes data.DataNodes = js.DataNodes data.MaxNodeID = js.MaxNodeID + data.FreezedDataNodes = js.FreezedDataNodes return data.Data.UnmarshalBinary(js.Data) } @@ -346,18 +426,34 @@ func (data *Data) CreateShardGroup(database, policy string, timestamp time.Time) return nil } + // Don't create shard on freezed nodes + availableNodes := make([]meta.NodeInfo, 0, len(data.DataNodes)) + freezedNodes := make(map[uint64]bool) + for _, n := range data.FreezedDataNodes { + freezedNodes[n] = true + } + for _, n := range data.DataNodes { + if freezedNodes[n.ID] { + continue + } + availableNodes = append(availableNodes, n) + } + // Require at least one replica but no more replicas than nodes. replicaN := rpi.ReplicaN if replicaN == 0 { replicaN = 1 - } else if replicaN > len(data.DataNodes) { - replicaN = len(data.DataNodes) + } else if replicaN > len(availableNodes) { + replicaN = len(availableNodes) + } + if replicaN < 1 { + return errors.New("No replica can be assigned") } // Determine shard count by node count divided by replication factor. // This will ensure nodes will get distributed across nodes evenly and // replicated the correct number of times. - shardN := len(data.DataNodes) / replicaN + shardN := len(availableNodes) / replicaN // Create the shard group. data.MaxShardGroupID++ @@ -379,11 +475,11 @@ func (data *Data) CreateShardGroup(database, policy string, timestamp time.Time) // Assign data nodes to shards via round robin. // Start from a repeatably "random" place in the node list. - nodeIndex := int(data.Index % uint64(len(data.DataNodes))) + nodeIndex := int(data.Index % uint64(len(availableNodes))) for i := range sgi.Shards { si := &sgi.Shards[i] for j := 0; j < replicaN; j++ { - nodeID := data.DataNodes[nodeIndex%len(data.DataNodes)].ID + nodeID := availableNodes[nodeIndex%len(availableNodes)].ID si.Owners = append(si.Owners, meta.ShardOwner{NodeID: nodeID}) nodeIndex++ } @@ -440,3 +536,24 @@ func (data *Data) RemoveShardOwner(id, nodeID uint64) { } } } + +// DeleteShardGroup removes a shard group from a database and retention policy by id. +func (data *Data) DeleteShardGroup(database, policy string, id uint64, t time.Time) error { + // Find retention policy. + rpi, err := data.RetentionPolicy(database, policy) + if err != nil { + return err + } else if rpi == nil { + return influxdb.ErrRetentionPolicyNotFound(policy) + } + + // Find shard group by ID and set its deletion timestamp. + for i := range rpi.ShardGroups { + if rpi.ShardGroups[i].ID == id { + rpi.ShardGroups[i].DeletedAt = t.UTC() + return nil + } + } + + return meta.ErrShardGroupNotFound +} diff --git a/services/meta/data_inner_test.go b/services/meta/data_inner_test.go new file mode 100644 index 0000000..f0b0a64 --- /dev/null +++ b/services/meta/data_inner_test.go @@ -0,0 +1,24 @@ +package meta + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestNewShardOwner(t *testing.T) { + assert.Equal(t, uint64(0), newShardOwner(nil)) + assert.Equal(t, uint64(0), newShardOwner(map[uint64]int{})) + assert.Equal(t, uint64(1), newShardOwner(map[uint64]int{ + 5: 1, + 7: 1, + 1: 1, + 3: 1, + })) + assert.Equal(t, uint64(5), newShardOwner(map[uint64]int{ + 5: 1, + 7: 2, + 1: 2, + 3: 2, + })) +} diff --git a/services/meta/data_test.go b/services/meta/data_test.go index fc35c76..c5bd8b4 100644 --- a/services/meta/data_test.go +++ b/services/meta/data_test.go @@ -5,6 +5,7 @@ import ( "time" "github.com/influxdata/influxdb/services/meta" + "github.com/stretchr/testify/assert" imeta "github.com/angopher/chronus/services/meta" ) @@ -18,54 +19,93 @@ func newData() *imeta.Data { } } +func initialTwoDataNodes(data *imeta.Data) (uint64, uint64) { + host := "127.0.0.1:8080" + tcpHost := "127.0.0.1:8081" + id1, _ := data.CreateDataNode(host, tcpHost) + + host = "127.0.0.1:9080" + tcpHost = "127.0.0.1:9081" + id2, _ := data.CreateDataNode(host, tcpHost) + + return id1, id2 +} + func TestCreateDataNode(t *testing.T) { data := newData() host := "127.0.0.1:8080" tcpHost := "127.0.0.1:8081" - if err := data.CreateDataNode(host, tcpHost); err != nil { - t.Fatalf("unexpected err: %s", err) - } - expN := meta.NodeInfo{ID: 1, TCPHost: tcpHost, Host: host} - n := data.DataNode(1) - if n.ID != expN.ID || n.TCPHost != expN.TCPHost || n.Host != expN.Host { - t.Fatalf("expected node: %+v, got: %+v", expN, n) - } + // create one node + id, err := data.CreateDataNode(host, tcpHost) + assert.Nil(t, err) + assert.True(t, id > 0) - if err, exp := data.CreateDataNode(host, tcpHost), imeta.ErrNodeExists; err != exp { - t.Fatalf("expected err: %s, got: %s", exp, err) - } + // get it by id + n := data.DataNode(id) + assert.NotNil(t, n) + assert.Equal(t, id, n.ID) + assert.Equal(t, host, n.Host) + assert.Equal(t, tcpHost, n.TCPHost) + + // refetch, ensure it should not affect inner store + n.Host = "test" + n = data.DataNode(id) + assert.NotNil(t, n) + assert.Equal(t, id, n.ID) + assert.Equal(t, host, n.Host) + assert.Equal(t, tcpHost, n.TCPHost) + + // duplicated creation + _, err = data.CreateDataNode(host, tcpHost) + assert.Equal(t, imeta.ErrNodeExists, err) + + // try to delete it (simple deletion) + err = data.DeleteDataNode(0) + assert.Equal(t, imeta.ErrNodeIDRequired, err) + err = data.DeleteDataNode(9999999) + assert.Equal(t, imeta.ErrNodeNotFound, err) + err = data.DeleteDataNode(id) + assert.Nil(t, err) + + // fetch after deletion + n = data.DataNode(id) + assert.Nil(t, n) } func TestCreateAndDeleteMetaNode(t *testing.T) { data := newData() host := "127.0.0.1:8080" tcpHost := "127.0.0.1:8081" - if err := data.CreateMetaNode(host, tcpHost); err != nil { - t.Fatalf("unexpected err: %s", err) - } - expN := meta.NodeInfo{ID: 1, TCPHost: tcpHost, Host: host} - n := data.MetaNodes[0] - if n.ID != expN.ID || n.TCPHost != expN.TCPHost || n.Host != expN.Host { - t.Fatalf("expected node: %+v, got: %+v", expN, n) - } + // create + id, err := data.CreateMetaNode(host, tcpHost) + assert.Nil(t, err) - if err, exp := data.CreateMetaNode(host, tcpHost), imeta.ErrNodeExists; err != exp { - t.Fatalf("expected err: %s, got: %s", exp, err) - } + // fetch back + n := data.MetaNode(id) + assert.NotNil(t, n) + assert.Equal(t, id, n.ID) + assert.Equal(t, host, n.Host) + assert.Equal(t, tcpHost, n.TCPHost) - if err, exp := data.DeleteMetaNode(0), imeta.ErrNodeIDRequired; err != exp { - t.Fatalf("expected err: %s, got: %s", exp, err) - } + // recreation + _, err = data.CreateMetaNode(host, tcpHost) + assert.Equal(t, imeta.ErrNodeExists, err) - if err, exp := data.DeleteMetaNode(2), imeta.ErrNodeNotFound; err != exp { - t.Fatalf("expected err: %s, got: %s", exp, err) - } + // try to delete it (simple deletion) + err = data.DeleteMetaNode(0) + assert.Equal(t, imeta.ErrNodeIDRequired, err) + err = data.DeleteMetaNode(9999999) + assert.Equal(t, imeta.ErrNodeNotFound, err) + err = data.DeleteMetaNode(id) + assert.Nil(t, err) + err = data.DeleteMetaNode(id) + assert.Equal(t, imeta.ErrNodeNotFound, err) - if err := data.DeleteMetaNode(expN.ID); err != nil { - t.Fatalf("unexpected err: %s", err) - } + // fetch after deletion + n = data.MetaNode(id) + assert.Nil(t, n) } func TestCreateShardGroup(t *testing.T) { @@ -75,27 +115,28 @@ func TestCreateShardGroup(t *testing.T) { data.CreateDatabase(name) data.CreateRetentionPolicy(name, meta.DefaultRetentionPolicyInfo(), true) - if err := data.CreateShardGroup(name, policy, time.Now()); err != imeta.ErrNodeNotFound { - t.Fatalf("expected err: %s, got: %s", imeta.ErrNodeNotFound, err) - } + // create group with no node in cluster + err := data.CreateShardGroup(name, policy, time.Now()) + assert.Equal(t, imeta.ErrNodeNotFound, err) - createTwoDataNode(data) + // add nodes to cluster + id1, id2 := initialTwoDataNodes(data) - if err := data.CreateShardGroup(name, policy, time.Now()); err != nil { - t.Fatalf("unexpected err: %s", err) - } + // recreation + err = data.CreateShardGroup(name, policy, time.Now()) + assert.Nil(t, err) rp := data.Database(name).RetentionPolicy(policy) + assert.NotNil(t, rp) + // replica = 1(default), shards = node_cnt / replica + assert.Equal(t, 1, len(rp.ShardGroups)) + assert.Equal(t, 2, len(rp.ShardGroups[0].Shards)) sgi := rp.ShardGroups[0] - if len(sgi.Shards) != 2 { - t.Fatalf("unexpected 2 shards, got: %d", len(sgi.Shards)) - } for _, sh := range sgi.Shards { + assert.Equal(t, 1, len(sh.Owners)) o := sh.Owners[0] - if len(sh.Owners) != 1 || o.NodeID != 1 && o.NodeID != 2 { - t.Fatalf("unexpected shard: %+v", sh) - } + assert.True(t, o.NodeID == id1 || o.NodeID == id2) } replicaN := 2 @@ -111,19 +152,15 @@ func TestCreateShardGroup(t *testing.T) { rp = data.Database(name).RetentionPolicy("rp") sgi = rp.ShardGroups[0] - if len(sgi.Shards) != 1 { - t.Fatalf("unexpected 1 shards, got: %d", len(sgi.Shards)) - } + assert.NotNil(t, rp) + assert.Equal(t, 1, len(rp.ShardGroups)) + assert.Equal(t, 1, len(rp.ShardGroups[0].Shards)) owners := sgi.Shards[0].Owners - if len(owners) != 2 { - t.Fatalf("unexpected shard: %+v", sgi.Shards[0]) - } + assert.Equal(t, 2, len(owners)) for _, o := range owners { - if o.NodeID != 1 && o.NodeID != 2 { - t.Fatalf("unexpected owner: %+v", o) - } + assert.True(t, o.NodeID == id1 || o.NodeID == id2) } } @@ -132,7 +169,7 @@ func TestDeleteDataNode(t *testing.T) { name := "testdb" policy := "rp" data.CreateDatabase(name) - createTwoDataNode(data) + id1, id2 := initialTwoDataNodes(data) replicaN := 2 duration := time.Hour @@ -145,28 +182,88 @@ func TestDeleteDataNode(t *testing.T) { data.CreateRetentionPolicy(name, spec.NewRetentionPolicyInfo(), true) data.CreateShardGroup(name, policy, time.Now()) - data.DeleteDataNode(1) - + // before deletion, 2 nodes rp := data.Database(name).RetentionPolicy("rp") + assert.NotNil(t, rp) + assert.Equal(t, 1, len(rp.ShardGroups)) sgi := rp.ShardGroups[0] - + assert.Equal(t, 1, len(sgi.Shards)) owners := sgi.Shards[0].Owners - if len(owners) != 1 { - t.Fatalf("unexpected shard: %+v", sgi.Shards[0]) - } + assert.Equal(t, 2, len(owners)) + + data.DeleteDataNode(id1) + + // delete 1 node + rp = data.Database(name).RetentionPolicy("rp") + assert.NotNil(t, rp) + assert.Equal(t, 1, len(rp.ShardGroups)) + sgi = rp.ShardGroups[0] + assert.Equal(t, 1, len(sgi.Shards)) + owners = sgi.Shards[0].Owners + assert.Equal(t, 1, len(owners)) o := owners[0] - if o.NodeID != 2 { - t.Fatalf("unexpected owner: %+v", o) - } + assert.Equal(t, id2, o.NodeID) } -func createTwoDataNode(data *imeta.Data) { - host := "127.0.0.1:8080" - tcpHost := "127.0.0.1:8081" - data.CreateDataNode(host, tcpHost) +func TestFreezeDataNode(t *testing.T) { + data := newData() + id1, id2 := initialTwoDataNodes(data) + replicaN := 2 + name := "testdb" + policy := "rp" + duration := time.Hour + data.CreateDatabase(name) + spec := meta.RetentionPolicySpec{ + Name: policy, + ReplicaN: &replicaN, + Duration: &duration, + ShardGroupDuration: time.Minute, + } + data.CreateRetentionPolicy(name, spec.NewRetentionPolicyInfo(), true) + data.CreateShardGroup(name, policy, time.Now()) + sg, err := data.ShardGroups(name, policy) + assert.Nil(t, err) + assert.Equal(t, 1, len(sg)) + assert.Equal(t, 2, len(sg[0].Shards[0].Owners)) - host = "127.0.0.1:9080" - tcpHost = "127.0.0.1:9081" - data.CreateDataNode(host, tcpHost) + assert.Equal(t, 0, len(data.FreezedDataNodes)) + + assert.NotNil(t, data.FreezeDataNode(3333)) + assert.Nil(t, data.FreezeDataNode(id1)) + assert.Nil(t, data.FreezeDataNode(id2)) + assert.Equal(t, []uint64{id1, id2}, data.FreezedDataNodes) + assert.True(t, data.IsFreezeDataNode(id1)) + assert.True(t, data.IsFreezeDataNode(id2)) + + assert.Nil(t, data.UnfreezeDataNode(id2)) + assert.Equal(t, []uint64{id1}, data.FreezedDataNodes) + + assert.Nil(t, data.UnfreezeDataNode(id1)) + assert.Equal(t, []uint64{}, data.FreezedDataNodes) + assert.Nil(t, data.FreezeDataNode(id1)) + assert.NotNil(t, data.FreezeDataNode(id1)) + assert.Equal(t, []uint64{id1}, data.FreezedDataNodes) + data.CreateShardGroup(name, policy, time.Now().Add(time.Hour)) + sg, err = data.ShardGroups(name, policy) + assert.Nil(t, err) + assert.Equal(t, 2, len(sg)) + assert.Equal(t, 1, len(sg[1].Shards[0].Owners)) + + // try delete freezed node + data.DeleteDataNode(1) + assert.Equal(t, []uint64{}, data.FreezedDataNodes) +} + +func TestClone(t *testing.T) { + data1 := newData() + id1, id2 := initialTwoDataNodes(data1) + data2 := data1.Clone() + data1.FreezeDataNode(id1) + data2.FreezeDataNode(id2) + data3 := data1.Clone() + assert.Equal(t, 1, len(data1.FreezedDataNodes)) + assert.Equal(t, 1, len(data2.FreezedDataNodes)) + assert.NotEqual(t, data1.FreezedDataNodes, data2.FreezedDataNodes) + assert.Equal(t, data3.FreezedDataNodes, data1.FreezedDataNodes) } diff --git a/services/meta/errors.go b/services/meta/errors.go index 486fa57..0097f62 100644 --- a/services/meta/errors.go +++ b/services/meta/errors.go @@ -11,6 +11,12 @@ var ( // ErrNodeNotFound is returned when mutating a node that doesn't exist. ErrNodeNotFound = errors.New("node not found") + // ErrNodeAlreadyFreezed represents node has been already freezed + ErrNodeAlreadyFreezed = errors.New("node has been freezed before") + + // ErrNodeNotFreezed represents node is not freezed (is normal) + ErrNodeNotFreezed = errors.New("node hasn't been freezed") + // ErrNodesRequired is returned when at least one node is required for an operation. // This occurs when creating a shard group. ErrNodesRequired = errors.New("at least one node required") diff --git a/services/meta/client.go b/services/meta/store.go similarity index 89% rename from services/meta/client.go rename to services/meta/store.go index d9f5a41..7378ad4 100644 --- a/services/meta/client.go +++ b/services/meta/store.go @@ -7,19 +7,14 @@ import ( crand "crypto/rand" "crypto/sha256" "errors" - "fmt" "io" - "io/ioutil" "net/http" - "os" - "path/filepath" "sort" "sync" "time" "github.com/influxdata/influxdb" "github.com/influxdata/influxdb/logger" - "github.com/influxdata/influxdb/pkg/file" "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxql" "go.uber.org/zap" @@ -27,14 +22,14 @@ import ( ) const ( - // SaltBytes is the number of bytes used for salts. - SaltBytes = 32 + // SALT_LENGTH is the number of bytes used for salts. + SALT_LENGTH = 32 - metaFile = "meta.db" + META_FILE = "meta.db" - // ShardGroupDeletedExpiration is the amount of time before a shard group info will be removed from cached + // SHARDGROUP_INFO_EVICTION is the amount of time before a shard group info will be removed from cached // data after it has been marked deleted (2 weeks). - ShardGroupDeletedExpiration = -2 * 7 * 24 * time.Hour + SHARDGROUP_INFO_EVICTION = -2 * 7 * 24 * time.Hour ) // Client is used to execute commands on and read data from @@ -79,10 +74,6 @@ func NewClient(config *meta.Config) *Client { } } -func (c *Client) Print() { - fmt.Printf("%+v\n", c.cacheData) -} - // Open a connection to a meta service cluster. func (c *Client) Open() error { c.mu.Lock() @@ -134,28 +125,30 @@ func (c *Client) data() *Data { return c.cacheData } -// Node returns a node by id. +// DataNode returns a node by id. +// If specified node doesn't exist a meta.ErrNodeNotFound error will be returned. func (c *Client) DataNode(id uint64) (*meta.NodeInfo, error) { c.mu.Lock() defer c.mu.Unlock() - for _, n := range c.data().DataNodes { - if n.ID == id { - return &n, nil - } + n := c.data().DataNode(id) + if n == nil { + return nil, ErrNodeNotFound } - return nil, ErrNodeNotFound + return n, nil } -// DataNodes returns the data nodes' info. -func (c *Client) DataNodes() ([]meta.NodeInfo, error) { - return c.data().DataNodes, nil +// DataNodes returns all nodes +func (c *Client) DataNodes() []meta.NodeInfo { + c.mu.Lock() + defer c.mu.Unlock() + return c.data().DataNodes } // CreateDataNode will create a new data node in the metastore func (c *Client) CreateDataNode(httpAddr, tcpAddr string) (*meta.NodeInfo, error) { c.mu.Lock() defer c.mu.Unlock() - err := c.data().CreateDataNode(httpAddr, tcpAddr) + _, err := c.data().CreateDataNode(httpAddr, tcpAddr) if err != nil { return nil, err } @@ -172,10 +165,11 @@ func (c *Client) CreateDataNode(httpAddr, tcpAddr string) (*meta.NodeInfo, error // DataNodeByHTTPHost returns the data node with the give http bind address func (c *Client) DataNodeByHTTPHost(httpAddr string) (*meta.NodeInfo, error) { - nodes, _ := c.DataNodes() + nodes := c.data().DataNodes for _, n := range nodes { if n.Host == httpAddr { - return &n, nil + newN := n + return &newN, nil } } @@ -184,10 +178,11 @@ func (c *Client) DataNodeByHTTPHost(httpAddr string) (*meta.NodeInfo, error) { // DataNodeByTCPHost returns the data node with the give http bind address func (c *Client) DataNodeByTCPHost(tcpAddr string) (*meta.NodeInfo, error) { - nodes, _ := c.DataNodes() + nodes := c.data().DataNodes for _, n := range nodes { if n.TCPHost == tcpAddr { - return &n, nil + newN := n + return &newN, nil } } @@ -198,11 +193,12 @@ func (c *Client) DataNodeByTCPHost(tcpAddr string) (*meta.NodeInfo, error) { func (c *Client) DeleteDataNode(id uint64) error { c.mu.Lock() defer c.mu.Unlock() - err := c.cacheData.DeleteDataNode(id) + data := c.cacheData.Clone() + err := data.DeleteDataNode(id) if err != nil { return err } - if err := c.commit(c.cacheData); err != nil { + if err := c.commit(data); err != nil { return err } return nil @@ -468,10 +464,6 @@ func (c *Client) user(name string) (meta.User, error) { return nil, meta.ErrUserNotFound } -// bcryptCost is the cost associated with generating password with bcrypt. -// This setting is lowered during testing to improve test suite performance. -var bcryptCost = bcrypt.DefaultCost - // hashWithSalt returns a salted hash of password using salt. func (c *Client) hashWithSalt(salt []byte, password string) []byte { hasher := sha256.New() @@ -482,7 +474,7 @@ func (c *Client) hashWithSalt(salt []byte, password string) []byte { // saltedHash returns a salt and salted hash of password. func (c *Client) saltedHash(password string) (salt, hash []byte, err error) { - salt = make([]byte, SaltBytes) + salt = make([]byte, SALT_LENGTH) if _, err := io.ReadFull(crand.Reader, salt); err != nil { return nil, nil, err } @@ -491,7 +483,7 @@ func (c *Client) saltedHash(password string) (salt, hash []byte, err error) { } // CreateUser adds a user with the given name and password and admin status. -func (c *Client) CreateUser(name, password string, admin bool) (meta.User, error) { +func (c *Client) CreateUser(name, hashedPassword string, admin bool) (meta.User, error) { c.mu.Lock() defer c.mu.Unlock() @@ -499,19 +491,14 @@ func (c *Client) CreateUser(name, password string, admin bool) (meta.User, error // See if the user already exists. if u, err := c.user(name); err != nil && u != nil { - if err := bcrypt.CompareHashAndPassword([]byte(u.(*meta.UserInfo).Hash), []byte(password)); err != nil || u.(*meta.UserInfo).Admin != admin { + info := u.(*meta.UserInfo) + if info.Hash != hashedPassword || info.Admin != admin { return nil, meta.ErrUserExists } return u, nil } - // Hash the password before serializing it. - hash, err := bcrypt.GenerateFromPassword([]byte(password), bcryptCost) - if err != nil { - return nil, err - } - - if err := data.CreateUser(name, string(hash), admin); err != nil { + if err := data.CreateUser(name, hashedPassword, admin); err != nil { return nil, err } @@ -523,23 +510,17 @@ func (c *Client) CreateUser(name, password string, admin bool) (meta.User, error } // UpdateUser updates the password of an existing user. -func (c *Client) UpdateUser(name, password string) error { +func (c *Client) UpdateUser(name, hashedPassword string) error { c.mu.Lock() defer c.mu.Unlock() data := c.cacheData.Clone() - // Hash the password before serializing it. - hash, err := bcrypt.GenerateFromPassword([]byte(password), bcryptCost) - if err != nil { + if err := data.UpdateUser(name, hashedPassword); err != nil { return err } - if err := data.UpdateUser(name, string(hash)); err != nil { - return err - } - - delete(c.authCache, name) + defer delete(c.authCache, name) return c.commit(data) } @@ -555,6 +536,8 @@ func (c *Client) DropUser(name string) error { return err } + defer delete(c.authCache, name) + if err := c.commit(data); err != nil { return err } @@ -783,9 +766,8 @@ func (c *Client) TruncateShardGroups(t time.Time) error { } // PruneShardGroups remove deleted shard groups from the data store. -func (c *Client) PruneShardGroups() error { +func (c *Client) PruneShardGroups(expiration time.Time) error { var changed bool - expiration := time.Now().Add(ShardGroupDeletedExpiration) c.mu.Lock() defer c.mu.Unlock() data := c.cacheData.Clone() @@ -870,14 +852,58 @@ func createShardGroup(data *Data, database, policy string, timestamp time.Time) return sgi, nil } +// IsDataNodeFreezed returns whether the node has been freezed +func (c *Client) IsDataNodeFreezed(id uint64) bool { + c.mu.Lock() + defer c.mu.Unlock() + + return c.cacheData.IsFreezeDataNode(id) +} + +// FreezeDataNode freezes specific node for new shard's creation +func (c *Client) FreezeDataNode(id uint64) error { + c.mu.Lock() + defer c.mu.Unlock() + + data := c.cacheData.Clone() + + if err := data.FreezeDataNode(id); err != nil { + return err + } + + if err := c.commit(data); err != nil { + return err + } + + return nil +} + +// UnfreezeDataNode restores specific node for new shard's creation +func (c *Client) UnfreezeDataNode(id uint64) error { + c.mu.Lock() + defer c.mu.Unlock() + + data := c.cacheData.Clone() + + if err := data.UnfreezeDataNode(id); err != nil { + return err + } + + if err := c.commit(data); err != nil { + return err + } + + return nil +} + // DeleteShardGroup removes a shard group from a database and retention policy by id. -func (c *Client) DeleteShardGroup(database, policy string, id uint64) error { +func (c *Client) DeleteShardGroup(database, policy string, id uint64, t time.Time) error { c.mu.Lock() defer c.mu.Unlock() data := c.cacheData.Clone() - if err := data.DeleteShardGroup(database, policy, id); err != nil { + if err := data.DeleteShardGroup(database, policy, id, t); err != nil { return err } @@ -1045,20 +1071,15 @@ func (c *Client) DropSubscription(database, rp, name string) error { // SetData overwrites the underlying data in the meta store. func (c *Client) SetData(data *Data) error { c.mu.Lock() + defer c.mu.Unlock() // reset the index so the commit will fire a change event c.cacheData.Index = 0 - // increment the index to force the changed channel to fire - d := data.Clone() - d.Index++ - - if err := c.commit(d); err != nil { + if err := c.commit(data.Clone()); err != nil { return err } - c.mu.Unlock() - return nil } @@ -1137,63 +1158,13 @@ func (c *Client) WithLogger(log *zap.Logger) { // snapshot saves the current meta data to disk. func snapshot(path string, data *Data) error { - //TODO: no need write snapshot to disk + // no need write snapshot to disk return nil - filename := filepath.Join(path, metaFile) - tmpFile := filename + "tmp" - - f, err := os.Create(tmpFile) - if err != nil { - return err - } - defer f.Close() - - var d []byte - if b, err := data.MarshalBinary(); err != nil { - return err - } else { - d = b - } - - if _, err := f.Write(d); err != nil { - return err - } - - if err = f.Sync(); err != nil { - return err - } - - //close file handle before renaming to support Windows - if err = f.Close(); err != nil { - return err - } - - return file.RenameFile(tmpFile, filename) } // Load loads the current meta data from disk. func (c *Client) Load() error { - //TODO:no need load - return nil - file := filepath.Join(c.path, metaFile) - - f, err := os.Open(file) - if err != nil { - if os.IsNotExist(err) { - return nil - } - return err - } - defer f.Close() - - data, err := ioutil.ReadAll(f) - if err != nil { - return err - } - - if err := c.cacheData.UnmarshalBinary(data); err != nil { - return err - } + // no need load return nil } diff --git a/services/meta/client_test.go b/services/meta/store_test.go similarity index 98% rename from services/meta/client_test.go rename to services/meta/store_test.go index 7dc0b65..4a9072d 100644 --- a/services/meta/client_test.go +++ b/services/meta/store_test.go @@ -14,6 +14,7 @@ import ( "github.com/influxdata/influxdb" "github.com/influxdata/influxdb/services/meta" "github.com/influxdata/influxql" + "golang.org/x/crypto/bcrypt" ) func TestMetaClient_CreateDatabaseOnly(t *testing.T) { @@ -575,6 +576,11 @@ func TestMetaClient_DropRetentionPolicy(t *testing.T) { } } +func hashPassword(password string) string { + hash, _ := bcrypt.GenerateFromPassword([]byte(password), bcrypt.DefaultCost) + return string(hash) +} + func TestMetaClient_CreateUser(t *testing.T) { t.Parallel() @@ -583,12 +589,12 @@ func TestMetaClient_CreateUser(t *testing.T) { defer c.Close() // Create an admin user - if _, err := c.CreateUser("fred", "supersecure", true); err != nil { + if _, err := c.CreateUser("fred", hashPassword("supersecure"), true); err != nil { t.Fatal(err) } // Create a non-admin user - if _, err := c.CreateUser("wilma", "password", false); err != nil { + if _, err := c.CreateUser("wilma", hashPassword("password"), false); err != nil { t.Fatal(err) } @@ -621,7 +627,7 @@ func TestMetaClient_CreateUser(t *testing.T) { } // Change password should succeed. - if err := c.UpdateUser("fred", "moresupersecure"); err != nil { + if err := c.UpdateUser("fred", hashPassword("moresupersecure")); err != nil { t.Fatal(err) } @@ -1083,7 +1089,7 @@ func TestMetaClient_PruneShardGroups(t *testing.T) { t.Fatal(err) } - if err := c.PruneShardGroups(); err != nil { + if err := c.PruneShardGroups(time.Now().Add(imeta.SHARDGROUP_INFO_EVICTION)); err != nil { t.Fatal(err) } diff --git a/services/migrate/executor.go b/services/migrate/executor.go new file mode 100644 index 0000000..d59013c --- /dev/null +++ b/services/migrate/executor.go @@ -0,0 +1,177 @@ +package migrate + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "os" + "path/filepath" + "time" + + "github.com/angopher/chronus/x" + "github.com/influxdata/influxdb/pkg/tar" + "github.com/influxdata/influxdb/services/meta" + "github.com/influxdata/influxdb/services/snapshotter" + "github.com/influxdata/influxdb/tcp" + "github.com/influxdata/influxdb/tsdb" + "go.uber.org/zap" +) + +const ( + MAX_RETRY = 3 +) + +type ValidatorFn func(file string) error +type ProgressCB func(copied uint64) + +type ManagerInterface interface { + Add(task *Task) error + Remove(id uint64) + Tasks() []*Task + Kill(id uint64, source, dest string) +} + +type MetaClientInterface interface { + ShardOwner(shardID uint64) (database, policy string, sgi *meta.ShardGroupInfo) + AddShardOwner(shardID, nodeID uint64) error +} + +type TSDBInterface interface { + Path() string + ShardRelativePath(id uint64) (string, error) + CreateShard(database, retentionPolicy string, shardID uint64, enabled bool) error + Shard(id uint64) *tsdb.Shard +} + +func (m *Manager) execute(t *Task) (err error) { + t.Started = true + t.StartTime = time.Now().Unix() + defer func() { + if !t.Finished { + t.error(fmt.Errorf("Unknow error for task on shard %d", t.ShardId)) + } + }() + i := 1 + delay := 2 * time.Second + for i <= MAX_RETRY { + err = m.replicate(t) + if err == nil { + return + } + time.Sleep(delay) + // backoff by 2 + delay *= 2 + i++ + } + return fmt.Errorf("Migration for shard %d failed after retries", t.ShardId) +} + +func backupFromRemote(t *Task, logger *zap.SugaredLogger) (*os.File, error) { + // prepare local tmp file + tmpFilePath := filepath.Join(t.TmpStorePath, fmt.Sprint("shard_", t.ShardId)) + tmpFile, err := os.Create(tmpFilePath) + if err != nil { + return nil, fmt.Errorf("open temp file failed: %s", err) + } + defer tmpFile.Close() + // Connect to snapshotter service. + conn, err := tcp.Dial("tcp", t.SrcHost, snapshotter.MuxHeader) + if err != nil { + return tmpFile, err + } + defer conn.Close() + logger.Info("Connected to ", t.SrcHost) + + req := &snapshotter.Request{ + Type: snapshotter.RequestShardBackup, + BackupDatabase: t.Database, + BackupRetentionPolicy: t.Retention, + ShardID: t.ShardId, + } + + // Write the request type + _, err = conn.Write([]byte{byte(req.Type)}) + if err != nil { + return tmpFile, err + } + + // Write the request + if err := json.NewEncoder(conn).Encode(req); err != nil { + return tmpFile, fmt.Errorf("encode snapshot request: %s", err) + } + + buf := make([]byte, 1024*1024) + n := 0 + ctx := context.Background() + for { + t.Limiter.Allow() + n, err = conn.Read(buf) + if err != nil { + break + } + if n == 0 { + break + } + tmpFile.Write(buf[:n]) + t.Copied += uint64(n) + // limiter in post way + t.Limiter.WaitN(ctx, n) + if t.ProgressLimiter.Allow() { + logger.Infof("Migrating shard %d: %d copied", t.ShardId, t.Copied) + } + } + logger.Infof("Transfer Shard %d completely with %d bytes", t.ShardId, t.Copied) + + // XXX: Verify + return tmpFile, nil +} + +func (m *Manager) replicate(t *Task) error { + if x.Exists(t.DstStorePath) != x.NotExisted { + t.error(errors.New("Destination shard directory has already existed")) + return nil + } + if x.Exists(t.TmpStorePath) == x.NotExisted { + t.error(errors.New("Temporary directory is not existed")) + return nil + } + + // transfer to local + m.logger.Infof("start to execute task copying %d from %s", t.ShardId, t.SrcHost) + tmpFile, err := backupFromRemote(t, m.logger) + defer func() { + // remove tmp file + if tmpFile != nil { + os.Remove(tmpFile.Name()) + } + }() + if err != nil { + return err + } + + // unpack to destination + os.MkdirAll(t.DstStorePath, os.FileMode(0755)) + err = restoreFromTar(tmpFile.Name(), t.DstStorePath) + if err != nil { + m.logger.Errorf("Unpack from backup failed for shard %d: %v", t.ShardId, err) + // remove incorrect data + os.RemoveAll(t.DstStorePath) + return err + } + + t.succ() + return nil +} + +// restore restores a tar archive to the target path +func restoreFromTar(tarFile, path string) error { + f, err := os.Open(tarFile) + if err != nil { + return err + } + defer f.Close() + + os.MkdirAll(path, 0755) + return tar.Restore(f, path) +} diff --git a/services/migrate/executor_test.go b/services/migrate/executor_test.go new file mode 100644 index 0000000..a95f296 --- /dev/null +++ b/services/migrate/executor_test.go @@ -0,0 +1,61 @@ +package migrate + +import ( + "os" + "path/filepath" + "testing" + + "github.com/influxdata/influxdb/pkg/tar" + "github.com/stretchr/testify/assert" +) + +func TestExecutorRestore(t *testing.T) { + basePath := os.TempDir() + ".chronus/tmp" + tarFilePath := os.TempDir() + ".chronus/t.tar" + os.RemoveAll(basePath) + assert.Nil(t, os.MkdirAll(basePath, os.FileMode(0755))) + defer os.RemoveAll(basePath) + + // initial files: a, b + f, err := os.Create(basePath + "/a") + assert.Nil(t, err) + f.WriteString("testA") + f.Close() + + f, err = os.Create(basePath + "/b") + assert.Nil(t, err) + f.WriteString("testB") + f.Close() + + os.Remove(tarFilePath) + f, err = os.Create(tarFilePath) + assert.Nil(t, err) + defer os.Remove(tarFilePath) + + // create tar file + err = tar.Stream(f, basePath, "/db/rp/1", nil) + assert.Nil(t, err) + f.Close() + + // verify files before removing + arr, err := filepath.Glob(basePath + "/*") + assert.Nil(t, err) + assert.Equal(t, 2, len(arr)) + + // remove them + os.Remove(basePath + "/a") + os.Remove(basePath + "/b") + + // verify it's empty + arr, err = filepath.Glob(basePath + "/*") + assert.Nil(t, err) + assert.Equal(t, 0, len(arr)) + + // extract tar file + assert.Nil(t, restoreFromTar(tarFilePath, basePath)) + + // verify it's comtent + arr, err = filepath.Glob(basePath + "/1/*") + assert.Nil(t, err) + assert.Equal(t, 2, len(arr)) +} diff --git a/services/migrate/manager.go b/services/migrate/manager.go new file mode 100644 index 0000000..2091053 --- /dev/null +++ b/services/migrate/manager.go @@ -0,0 +1,215 @@ +package migrate + +import ( + "errors" + "io" + "sync" + + "go.uber.org/zap" + "golang.org/x/time/rate" +) + +const ( + TASK_PARALLEL_MAX = 24 +) + +var ( + ErrTaskDuplicated = errors.New("Shard migration task has already existed") +) + +// Task holds the task info +type Task struct { + // State + Started bool + Finished bool + StartTime int64 // timestamp in second + Error error + C chan error + + // Meta + Database string + Retention string + ShardId uint64 + SrcHost string + + // Store + DstStorePath string + TmpStorePath string + + // Progress + Copied uint64 + Limiter *rate.Limiter + ProgressLimiter *rate.Limiter // for progress logging + + // Callback + Closer io.Closer +} + +func (t *Task) succ() { + t.Finished = true + close(t.C) + if t.Closer != nil { + t.Closer.Close() + } +} + +func (t *Task) error(err error) { + t.Error = err + t.Finished = true + t.C <- err + close(t.C) + if t.Closer != nil { + t.Closer.Close() + } +} + +// Manager is the container of tasks running or queued +// You should use NewCopyManager to get an instance +type Manager struct { + sync.Mutex + wg sync.WaitGroup + cond *sync.Cond + taskMap map[uint64]*Task + taskQueue []*Task + parallel int + logger *zap.SugaredLogger + shouldStop bool +} + +// NewManager creates and returns a new manager +func NewManager(parallel int) *Manager { + if parallel < 1 { + parallel = 1 + } + if parallel > TASK_PARALLEL_MAX { + parallel = TASK_PARALLEL_MAX + } + m := &Manager{ + parallel: parallel, + taskMap: make(map[uint64]*Task), + taskQueue: make([]*Task, 0, TASK_PARALLEL_MAX), + cond: sync.NewCond(&sync.Mutex{}), + logger: zap.NewNop().Sugar(), + } + return m +} + +func (m *Manager) Start() { + for i := 0; i < m.parallel; i++ { + go m.loop() + } + m.logger.Infof("%d migrators started", m.parallel) +} + +func (m *Manager) loop() { + m.wg.Add(1) + defer m.wg.Done() + for !m.shouldStop { + t := m.pop() + if t == nil { + m.cond.L.Lock() + m.cond.Wait() + m.cond.L.Unlock() + continue + } + + m.execute(t) + m.Remove(t.ShardId) + } +} + +func (m *Manager) Close() { + m.shouldStop = true + m.cond.Broadcast() + m.wg.Wait() + m.logger.Info("Shutdown migrating manager") +} + +func (m *Manager) WithLogger(log *zap.Logger) { + m.logger = log.With(zap.String("service", "migrate.Manager")).Sugar() +} + +func (m *Manager) pop() *Task { + m.Lock() + defer m.Unlock() + if len(m.taskQueue) == 0 { + return nil + } + t := m.taskQueue[len(m.taskQueue)-1] + m.taskQueue = m.taskQueue[:len(m.taskQueue)-1] + return t +} + +// Add registers a task under specific shard and returns error if there was one already. +func (m *Manager) Add(task *Task) error { + m.Lock() + defer m.Unlock() + if _, ok := m.taskMap[task.ShardId]; ok { + return ErrTaskDuplicated + } + if task.C == nil { + task.C = make(chan error, 1) + } + // Currently limit to 5MB/s, 10MB/s in burst + task.Limiter = rate.NewLimiter(5*1024*1024, 10*1024*1024) + task.ProgressLimiter = rate.NewLimiter(0.05, 1) // 1 log every 20 seconds + m.taskMap[task.ShardId] = task + m.taskQueue = append(m.taskQueue, task) + m.cond.Signal() + return nil +} + +// Remove removes the task under specific shard id WITHOUT stopping it +func (m *Manager) Remove(id uint64) { + m.Lock() + defer m.Unlock() + if t, ok := m.taskMap[id]; ok { + delete(m.taskMap, id) + m.removeFromQueue(t) + } +} + +// Tasks returns all tasks +func (m *Manager) Tasks() []*Task { + m.Lock() + defer m.Unlock() + tasks := make([]*Task, 0, len(m.taskMap)) + for _, t := range m.taskMap { + tasks = append(tasks, t) + } + return tasks +} + +func (m *Manager) removeFromQueue(t *Task) { + if t == nil { + return + } + pos := -1 + for i, item := range m.taskQueue { + if item == t { + pos = i + break + } + } + if pos >= 0 { + m.taskQueue = append(m.taskQueue[0:pos], m.taskQueue[pos+1:]...) + } +} + +// Kill kills the running task and remove it from list +// Host addresses are required for double check +func (m *Manager) Kill(id uint64, srcHost string) { + m.Lock() + defer m.Unlock() + t, ok := m.taskMap[id] + if !ok { + return + } + if srcHost == t.SrcHost { + if t.Closer != nil { + t.Closer.Close() + } + delete(m.taskMap, id) + m.removeFromQueue(t) + } +} diff --git a/services/migrate/manager_test.go b/services/migrate/manager_test.go new file mode 100644 index 0000000..b0fc637 --- /dev/null +++ b/services/migrate/manager_test.go @@ -0,0 +1,82 @@ +package migrate + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +type testCloser struct { + fn func() +} + +func (t testCloser) Close() error { + t.fn() + return nil +} + +func TestManagerNew(t *testing.T) { + m := NewManager(0) + assert.Equal(t, 1, m.parallel) + + m = NewManager(2) + assert.Equal(t, 2, m.parallel) + + m = NewManager(22222) + assert.Equal(t, TASK_PARALLEL_MAX, m.parallel) +} + +func TestManagerTasks(t *testing.T) { + cnt := 0 + task1 := &Task{ + ShardId: 123, + SrcHost: "a", + Closer: testCloser{ + fn: func() { + cnt++ + }, + }, + } + task2 := &Task{ + ShardId: 1235, + Closer: testCloser{ + fn: func() { + cnt++ + }, + }, + } + m := NewManager(1) + + assert.Equal(t, 0, len(m.Tasks())) + + // add task + assert.Nil(t, m.Add(task1)) + + assert.Equal(t, 1, len(m.Tasks())) + assert.Equal(t, task1, m.Tasks()[0]) + + // add another + assert.Nil(t, m.Add(task2)) + + assert.Equal(t, 2, len(m.Tasks())) + + assert.NotNil(t, m.pop()) + assert.NotNil(t, m.pop()) + assert.Nil(t, m.pop()) + + // kill + m.Kill(task1.ShardId, "") + assert.Equal(t, 2, len(m.Tasks())) + assert.Equal(t, 0, cnt) + m.Kill(task1.ShardId, "a") + assert.Equal(t, 1, len(m.Tasks())) + assert.Equal(t, 1, cnt) + + // remove only + m.Remove(task1.ShardId) + assert.Equal(t, 1, len(m.Tasks())) + assert.Equal(t, 1, cnt) + m.Remove(task2.ShardId) + assert.Equal(t, 0, len(m.Tasks())) + assert.Equal(t, 1, cnt) +} diff --git a/x/cyclic_buffer.go b/x/cyclic_buffer.go new file mode 100644 index 0000000..f397fca --- /dev/null +++ b/x/cyclic_buffer.go @@ -0,0 +1,101 @@ +package x + +// CyclicBuffer is a buffer which refills cyclically after writes. +// It can be compared to specific []byte efficiently. +// Well, it's NOT THREAD SAFE. +type CyclicBuffer struct { + buf []byte + pos int + cnt int +} + +func NewCyclicBuffer(capacity int) *CyclicBuffer { + return &CyclicBuffer{ + buf: make([]byte, capacity), + } +} + +func (c *CyclicBuffer) forward() { + c.pos++ + if c.pos >= len(c.buf) { + c.pos = 0 + } +} + +func (c *CyclicBuffer) write(data []byte, offset int) { + l := len(data) + for i := offset; i < l; i++ { + c.buf[c.pos] = data[i] + c.forward() + } +} + +func (c *CyclicBuffer) Cap() int { + return len(c.buf) +} + +func (c *CyclicBuffer) Len() int { + return c.cnt +} + +// Dump dumps internal data into given slice and returns write byte count +// ATTENTION: the given buffer should not smaller than buffer capacity +func (c *CyclicBuffer) Dump(data []byte) int { + l := len(c.buf) + if len(data) < l { + return 0 + } + posRead := c.pos + posWrite := 0 + skip := len(c.buf) - c.cnt + for i := 0; i < len(c.buf); i++ { + if skip > 0 { + skip-- + goto NEXT + } + data[posWrite] = c.buf[posRead] + posWrite++ + NEXT: + posRead++ + if posRead >= l { + posRead %= l + } + } + return posWrite +} + +func (c *CyclicBuffer) Write(data []byte) { + sz := len(data) + if sz < 1 { + return + } + c.cnt += sz + offset := sz - len(c.buf) + if offset < 0 { + offset = 0 + } + c.write(data, offset) +} + +// Compare returns true when matched +func (c *CyclicBuffer) Compare(data []byte) bool { + l0 := len(c.buf) + l1 := len(data) + if l0 != l1 { + return false + } + pos := c.pos + for i := 0; i < l1; i++ { + if data[i] != c.buf[pos] { + return false + } + pos++ + if pos >= l0 { + pos %= l0 + } + if pos == c.pos { + break + } + } + return true +} diff --git a/x/cyclic_buffer_test.go b/x/cyclic_buffer_test.go new file mode 100644 index 0000000..f91f433 --- /dev/null +++ b/x/cyclic_buffer_test.go @@ -0,0 +1,30 @@ +package x + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestCyclicBuffer(t *testing.T) { + buf := NewCyclicBuffer(4) + buf1 := []byte{0, 0, 0, 0} + buf.Write([]byte{}) + assert.False(t, buf.Compare([]byte{})) + assert.False(t, buf.Compare([]byte{0, 0, 0})) + assert.True(t, buf.Compare([]byte{0, 0, 0, 0})) + assert.Equal(t, 0, buf.Dump(buf1)) + assert.Equal(t, []byte{0, 0, 0, 0}, buf1) + buf.Write([]byte{1}) + assert.True(t, buf.Compare([]byte{0, 0, 0, 1})) + buf.Write([]byte{22}) + assert.True(t, buf.Compare([]byte{0, 0, 1, 22})) + assert.Equal(t, 2, buf.Dump(buf1)) + assert.Equal(t, []byte{1, 22}, buf1[:2]) + buf.Write([]byte{12, 33, 11}) + assert.True(t, buf.Compare([]byte{22, 12, 33, 11})) + buf.Write([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14}) + assert.Equal(t, 4, buf.Dump(buf1)) + assert.Equal(t, []byte{11, 12, 13, 14}, buf1) + assert.True(t, buf.Compare([]byte{11, 12, 13, 14})) +} diff --git a/x/echo_server.go b/x/echo_server.go new file mode 100644 index 0000000..d984659 --- /dev/null +++ b/x/echo_server.go @@ -0,0 +1,112 @@ +package x + +import ( + "io" + "net" + "sync" + "time" +) + +type EchoServer struct { + network string + address string + listener net.Listener + closing chan int + once sync.Once + wg sync.WaitGroup +} + +func NewEchoServer(network, addr string) *EchoServer { + return &EchoServer{ + network: network, + address: addr, + closing: make(chan int), + } +} + +func (s *EchoServer) Start() error { + listener, err := net.Listen(s.network, s.address) + if err != nil { + return err + } + s.listener = listener + go s.loop() + return nil +} + +func (s *EchoServer) Close() { + s.once.Do(func() { + close(s.closing) + s.listener.Close() + s.wg.Wait() + }) +} + +func (s *EchoServer) loop() { + defer s.Close() + + s.wg.Add(1) + defer s.wg.Done() +LOOP: + for { + select { + case <-s.closing: + break LOOP + default: + conn, err := s.listener.Accept() + if err != nil { + break LOOP + } + go s.connLoop(conn) + } + } +} + +func (s *EchoServer) connLoop(conn net.Conn) { + s.wg.Add(1) + defer func() { + conn.Close() + s.wg.Done() + }() + var ( + buf = make([]byte, 1024) + n int + err error + ) +LOOP: + for { + select { + case <-s.closing: + break LOOP + default: + conn.SetReadDeadline(time.Now().Add(100 * time.Millisecond)) + n, err = conn.Read(buf) + conn.SetDeadline(time.Time{}) + } + if err == io.EOF { + break LOOP + } + if err != nil { + continue + } + if n < 1 { + continue + } + if write(conn, buf[:n]) != nil { + break LOOP + } + } +} + +func write(conn net.Conn, data []byte) error { + size := len(data) + pos := 0 + for pos < size { + n, err := conn.Write(data[pos:size]) + if err != nil { + return err + } + pos += n + } + return nil +} diff --git a/x/io.go b/x/io.go new file mode 100644 index 0000000..7c40bf6 --- /dev/null +++ b/x/io.go @@ -0,0 +1,29 @@ +package x + +import ( + "os" +) + +type ExistStat int + +const ( + ExistUnknow ExistStat = iota + NotExisted + ExistFile + ExistDir +) + +// Exists returns the existed stat for specific path +func Exists(ospath string) ExistStat { + stat, err := os.Stat(ospath) + if err != nil { + if os.IsNotExist(err) { + return NotExisted + } + return ExistUnknow + } + if stat.IsDir() { + return ExistDir + } + return ExistFile +} diff --git a/x/io_test.go b/x/io_test.go new file mode 100644 index 0000000..68140a6 --- /dev/null +++ b/x/io_test.go @@ -0,0 +1,26 @@ +package x + +import ( + "os" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestExists(t *testing.T) { + assert.Equal(t, NotExisted, Exists("")) + assert.Equal(t, NotExisted, Exists("..........")) + assert.Equal(t, NotExisted, Exists("/asdf/asdf/asdf/sad/f/sadf/")) + assert.Equal(t, NotExisted, Exists("/asdf/asdf/asdf/sad/f/sadf/a")) + + tmp := os.TempDir() + "test/" + tmpFile := tmp + "a" + assert.Equal(t, NotExisted, Exists(tmp)) + assert.Equal(t, NotExisted, Exists(tmpFile)) + + os.MkdirAll(tmp, os.FileMode(0755)) + defer os.RemoveAll(tmp) + os.Create(tmpFile) + assert.Equal(t, ExistDir, Exists(tmp)) + assert.Equal(t, ExistFile, Exists(tmpFile)) +} diff --git a/x/math.go b/x/math.go new file mode 100644 index 0000000..e0f39fa --- /dev/null +++ b/x/math.go @@ -0,0 +1,51 @@ +package x + +import ( + "encoding/binary" + "math/rand" + "time" +) + +func Max(args ...int) int { + result := args[0] + for i := 1; i < len(args); i++ { + if args[i] > result { + result = args[i] + } + } + return result +} + +func Min(args ...int) int { + result := args[0] + for i := 1; i < len(args); i++ { + if args[i] < result { + result = args[i] + } + } + return result +} + +func fillOnce(data []byte, pos int) int { + var buf [8]byte + binary.BigEndian.PutUint64(buf[:], rand.Uint64()) + cnt := 0 + for pos+cnt < len(data) && cnt < len(buf) { + data[pos+cnt] = buf[cnt%8] + cnt++ + } + return cnt +} + +func RandBytes(n int) []byte { + if n < 1 { + return []byte{} + } + data := make([]byte, n) + pos := 0 + rand.Seed(time.Now().UnixNano()) + for pos < n { + pos += fillOnce(data, pos) + } + return data +} diff --git a/x/math_test.go b/x/math_test.go new file mode 100644 index 0000000..1de28e4 --- /dev/null +++ b/x/math_test.go @@ -0,0 +1,25 @@ +package x + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestMax(t *testing.T) { + assert.Equal(t, 0, Max(-1, 0, -33, 0, -99)) + assert.Equal(t, 3, Max(-1, 0, -33, 3, -99)) +} + +func TestMin(t *testing.T) { + assert.Equal(t, -99, Min(-1, 0, -33, 0, -99)) + assert.Equal(t, -33, Min(-1, 0, -33, 3, 99)) +} + +func TestRandBytes(t *testing.T) { + assert.Equal(t, 0, len(RandBytes(0))) + assert.Equal(t, 2, len(RandBytes(2))) + assert.Equal(t, 7, len(RandBytes(7))) + assert.Equal(t, 8, len(RandBytes(8))) + assert.Equal(t, 17, len(RandBytes(17))) +} diff --git a/x/pool.go b/x/pool.go new file mode 100644 index 0000000..8c40a3e --- /dev/null +++ b/x/pool.go @@ -0,0 +1,365 @@ +package x + +import ( + "errors" + "fmt" + "net" + "sync" + "sync/atomic" + "time" +) + +var ( + // ErrClosed is the error resulting if the pool is closed via pool.Close(). + ErrClosed = errors.New("pool is closed") +) + +type PoolStatistics struct { + Active, Idle, Capacity int + GetSuccessCnt, GetSuccessMillis uint64 + GetFailureCnt, GetFailureMillis uint64 + ReturnCnt, CloseCnt uint64 +} + +type ConnPool interface { + // Get returns a new connection from the pool. Closing the connections puts + // it back to the Pool. Closing it when the pool is destroyed or full will + // be counted as an error. + Get() (PooledConn, error) + + // Close closes the pool and all its connections. After Close() the pool is + // no longer usable. + Close() + + // Len returns current idled connection count in pool + Len() int + // Total returns total connection count managed by pool + Total() int + Statistics() PoolStatistics +} + +type PooledConn interface { + net.Conn + MarkUnusable() +} + +// boundedPool implements the Pool interface based on buffered channels. +type boundedPool struct { + // storage for our net.Conn connections + mu sync.RWMutex + // Using a channel to hold idled connections in pool is not a good idea + // for closing idled ones later. We may change it into an array to make + // more senses. + conns chan *pooledConn + + closer chan int + wg sync.WaitGroup + + waitTimeout time.Duration + idleTimeout time.Duration + initialCnt int + total int32 + // net.Conn generator + factory Factory + + // statistics + getSuccessCnt, getSuccessCost uint64 // in us + getFailureCnt, getFailureCost uint64 // in us + returnCnt, closeCnt uint64 +} + +// Factory is a function to create new connections. +type Factory func() (net.Conn, error) + +// NewBoundedPool returns a new pool based on buffered channels with an initial +// capacity, maximum capacity and waitTimeout to wait for a connection from the pool. +// Factory is used when initial capacity is +// greater than zero to fill the pool. A zero initialCap doesn't fill the Pool +// until a new Get() is called. During a Get(), If there is no new connection +// available in the pool and total connections is less than the max, a new connection +// will be created via the Factory() method. Othewise, the call will block until +// a connection is available or the waitTimeout is reached. +func NewBoundedPool(initialCnt, maxCap int, idleTimeout time.Duration, waitTimeout time.Duration, factory Factory) (ConnPool, error) { + if initialCnt < 0 || maxCap <= 0 || initialCnt > maxCap { + return nil, errors.New("invalid capacity settings") + } + + pool := &boundedPool{ + closer: make(chan int), + conns: make(chan *pooledConn, maxCap), + factory: factory, + initialCnt: initialCnt, + waitTimeout: waitTimeout, + idleTimeout: idleTimeout, + } + + // create initial connections, if something goes wrong, + // just close the pool error out. + for i := 0; i < initialCnt; i++ { + conn, err := factory() + if err != nil { + pool.Close() + return nil, fmt.Errorf("factory is not able to fill the pool: %s", err) + } + pool.conns <- pool.wrapConn(conn) + atomic.AddInt32(&pool.total, 1) + } + + go pool.idleChecker() + + return pool, nil +} + +func (pool *boundedPool) checkOnce() { + now := time.Now() + for int(atomic.LoadInt32(&pool.total)) > pool.initialCnt && len(pool.conns) > 0 { + select { + case conn := <-pool.conns: + if now.Sub(conn.lastUse) <= pool.idleTimeout { + select { + case pool.conns <- conn: + default: + // can't be returned, drop + conn.MarkUnusable() + conn.Close() + } + return + } + // drop it + conn.MarkUnusable() + conn.Close() + default: + return + } + } + +} + +// idleChecker checks the connections in pool and closes those haven't been used for idleTimeout and more than initialCnt. +func (pool *boundedPool) idleChecker() { + pool.wg.Add(1) + defer pool.wg.Done() + ticker := time.NewTicker(60 * time.Second) + defer ticker.Stop() +LOOP: + for { + select { + case <-ticker.C: + pool.checkOnce() + case <-pool.closer: + break LOOP + } + } +} + +// Get implements the Pool interfaces Get() method. If there is no new +// connection available in the pool, a new connection will be created via the +// Factory() method. +func (pool *boundedPool) Get() (newConn PooledConn, errOccurred error) { + conns := pool.conns + if conns == nil { + errOccurred = ErrClosed + return + } + + // statistics + begin := time.Now() + defer func() { + cost := uint64(time.Now().Sub(begin).Microseconds()) + if errOccurred == nil { + atomic.AddUint64(&pool.getSuccessCnt, 1) + atomic.AddUint64(&pool.getSuccessCost, cost) + } else if errOccurred != ErrClosed { + atomic.AddUint64(&pool.getFailureCnt, 1) + atomic.AddUint64(&pool.getFailureCost, cost) + } + }() + + // Try and grab a connection from the pool + select { + case conn := <-conns: + if conn == nil { + errOccurred = ErrClosed + return + } + newConn = conn + return + default: + // Could not get connection, can we create a new one? + total := atomic.LoadInt32(&pool.total) + capacity := int32(cap(conns)) + if total < capacity && atomic.AddInt32(&pool.total, 1) <= capacity { + conn, err := pool.factory() + if err != nil { + atomic.AddInt32(&pool.total, -1) + errOccurred = err + return + } + newConn = pool.wrapConn(conn) + return + } + } + + // The pool was empty and we couldn't create a new one to + // retry until one is free or we timeout + select { + case conn := <-conns: + if conn == nil { + errOccurred = ErrClosed + return + } + newConn = conn + return + case <-time.After(pool.waitTimeout): + errOccurred = fmt.Errorf("timed out waiting for free connection") + return + } +} + +// put puts the connection back to the pool. If the pool is full or closed, +// conn is simply closed. A nil conn will be rejected. +func (pool *boundedPool) put(conn *pooledConn) error { + if conn == nil { + return errors.New("connection is nil. rejecting") + } + + pool.mu.RLock() + defer pool.mu.RUnlock() + + if pool.conns == nil { + // pool is closed, close passed connection + goto DROP + } + + // put the resource back into the pool. If the pool is full, drop it + select { + case pool.conns <- conn: + return nil + default: + // pool is full, close passed connection + goto DROP + } + +DROP: + conn.MarkUnusable() + return conn.close() +} + +func closeIdleConnWhenShutdown(conn *pooledConn) { + conn.MarkUnusable() + conn.mu.Lock() + defer conn.mu.Unlock() + conn.close() +} + +func (pool *boundedPool) Close() { + close(pool.closer) + pool.wg.Wait() + pool.mu.Lock() + conns := pool.conns + pool.conns = nil + pool.factory = nil + pool.mu.Unlock() + + if conns == nil { + return + } + + close(conns) + for { + conn := <-conns + if conn == nil { + break + } + closeIdleConnWhenShutdown(conn) + } +} + +// Len returns current idled connection count in pool +func (pool *boundedPool) Len() int { + return len(pool.conns) +} + +// Total returns total connection count managed by pool +func (pool *boundedPool) Total() int { + return int(atomic.LoadInt32(&pool.total)) +} + +// newConn wraps a standard net.Conn to a poolConn net.Conn. +func (pool *boundedPool) wrapConn(conn net.Conn) *pooledConn { + c := &pooledConn{pool: pool} + c.Conn = conn + c.lastUse = time.Now() + return c +} + +func (pool *boundedPool) Statistics() PoolStatistics { + stat := PoolStatistics{} + stat.Capacity = cap(pool.conns) + stat.Idle = len(pool.conns) + total := atomic.LoadInt32(&pool.total) + stat.Active = int(total) - stat.Idle + + stat.GetSuccessCnt = atomic.LoadUint64(&pool.getSuccessCnt) + stat.GetSuccessMillis = atomic.LoadUint64(&pool.getSuccessCost) / 1e3 // us to ms + stat.GetFailureCnt = atomic.LoadUint64(&pool.getFailureCnt) + stat.GetFailureMillis = atomic.LoadUint64(&pool.getFailureCost) / 1e3 // us to ms + stat.ReturnCnt = atomic.LoadUint64(&pool.returnCnt) + stat.CloseCnt = atomic.LoadUint64(&pool.closeCnt) + + return stat +} + +// pooledConn is a wrapper around net.Conn to modify the the behavior of +// net.Conn's Close() method. +type pooledConn struct { + net.Conn + mu sync.Mutex + pool *boundedPool + lastUse time.Time + unusable bool +} + +// Close() puts the given connects back to the pool instead of closing it. +func (c *pooledConn) Close() error { + c.mu.Lock() + defer c.mu.Unlock() + return c.close() +} + +// inner close method, no MUTEX lock inside +func (c *pooledConn) close() error { + if c.Conn == nil { + return nil + } + defer func() { + c.Conn = nil + }() + + if c.unusable { + if c.pool != nil { + atomic.AddUint64(&c.pool.closeCnt, 1) + } + return c.Conn.Close() + } + if c.pool != nil { + atomic.AddUint64(&c.pool.returnCnt, 1) + } + return c.pool.put(&pooledConn{ + Conn: c.Conn, + pool: c.pool, + lastUse: time.Now(), + unusable: false, + }) +} + +// MarkUnusable() marks the connection not usable any more, to let the pool close it instead of returning it to pool. +func (c *pooledConn) MarkUnusable() { + c.mu.Lock() + defer c.mu.Unlock() + if c.unusable { + return + } + c.unusable = true + atomic.AddInt32(&c.pool.total, -1) +} diff --git a/x/pool_test.go b/x/pool_test.go new file mode 100644 index 0000000..e561155 --- /dev/null +++ b/x/pool_test.go @@ -0,0 +1,141 @@ +package x + +import ( + "fmt" + "net" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func dumpStatistics(stat PoolStatistics) { + fmt.Println("Active/Idle/Capacity:", fmt.Sprintf("%d/%d/%d", stat.Active, stat.Idle, stat.Capacity)) + fmt.Println("Get Success:", fmt.Sprintf("%d times, %d ms", stat.GetSuccessCnt, stat.GetSuccessMillis)) + fmt.Println("Get Failure:", fmt.Sprintf("%d times, %d ms", stat.GetFailureCnt, stat.GetFailureMillis)) + fmt.Println("Return:", stat.ReturnCnt) + fmt.Println("Close:", stat.CloseCnt) +} + +func TestPoolCapacity(t *testing.T) { + echoServer := NewEchoServer("tcp", "127.0.0.1:12345") + err := echoServer.Start() + assert.Nil(t, err) + cnt := 0 + pool, err := NewBoundedPool(1, 3, time.Second, time.Second, func() (net.Conn, error) { + cnt++ + return net.Dial("tcp", "127.0.0.1:12345") + }) + assert.Equal(t, 1, pool.Len()) + assert.Equal(t, 1, pool.Total()) + assert.Equal(t, 1, cnt) + + // normal get + conn1, err := pool.Get() + (pool.(*boundedPool)).checkOnce() + assert.Nil(t, err) + assert.Equal(t, 0, pool.Len()) + assert.Equal(t, 1, pool.Total()) + assert.Equal(t, 1, cnt) + + // normal get + conn2, err := pool.Get() + (pool.(*boundedPool)).checkOnce() + assert.Nil(t, err) + assert.Equal(t, 0, pool.Len()) + assert.Equal(t, 2, pool.Total()) + assert.Equal(t, 2, cnt) + + // normal get + conn3, err := pool.Get() + (pool.(*boundedPool)).checkOnce() + assert.Nil(t, err) + assert.Equal(t, 0, pool.Len()) + assert.Equal(t, 3, pool.Total()) + assert.Equal(t, 3, cnt) + + // cannot get more, failure + 1 + _, err = pool.Get() + (pool.(*boundedPool)).checkOnce() + assert.NotNil(t, err) + assert.Equal(t, 0, pool.Len()) + assert.Equal(t, 3, pool.Total()) + assert.Equal(t, 3, cnt) + + stat := pool.Statistics() + assert.Equal(t, 3, stat.Active) + assert.Equal(t, 3, stat.Capacity) + assert.Equal(t, uint64(3), stat.GetSuccessCnt) + assert.Equal(t, uint64(1), stat.GetFailureCnt) + assert.Equal(t, uint64(0), stat.ReturnCnt) + assert.Equal(t, uint64(0), stat.CloseCnt) + + // close 1 connection + conn3.Close() + (pool.(*boundedPool)).checkOnce() + assert.Equal(t, 1, pool.Len()) + assert.Equal(t, 3, pool.Total()) + assert.Equal(t, 3, cnt) + + // close remained 2 connections + conn2.Close() + conn1.Close() + (pool.(*boundedPool)).checkOnce() + assert.Equal(t, 3, pool.Len()) + assert.Equal(t, 3, pool.Total()) + assert.Equal(t, 3, cnt) + + time.Sleep(time.Second) + // recycle idled connections + (pool.(*boundedPool)).checkOnce() + (pool.(*boundedPool)).checkOnce() + (pool.(*boundedPool)).checkOnce() + (pool.(*boundedPool)).checkOnce() + assert.Equal(t, 1, pool.Len()) + assert.Equal(t, 1, pool.Total()) + + stat = pool.Statistics() + assert.Equal(t, 0, stat.Active) + assert.Equal(t, 3, stat.Capacity) + assert.Equal(t, uint64(3), stat.GetSuccessCnt) + assert.Equal(t, uint64(1), stat.GetFailureCnt) + assert.Equal(t, uint64(3), stat.ReturnCnt) + assert.Equal(t, uint64(2), stat.CloseCnt) + + pool.Close() + stat = pool.Statistics() + assert.Equal(t, uint64(3), stat.ReturnCnt) + assert.Equal(t, uint64(3), stat.CloseCnt) + echoServer.Close() +} + +func TestPoolCloseBeforeConnClose(t *testing.T) { + echoServer := NewEchoServer("tcp", "127.0.0.1:12345") + err := echoServer.Start() + assert.Nil(t, err) + cnt := 0 + pool, err := NewBoundedPool(1, 3, time.Second, time.Second, func() (net.Conn, error) { + cnt++ + return net.Dial("tcp", "127.0.0.1:12345") + }) + + // normal get + conn, err := pool.Get() + (pool.(*boundedPool)).checkOnce() + assert.Nil(t, err) + assert.Equal(t, 0, pool.Len()) + assert.Equal(t, 1, pool.Total()) + assert.Equal(t, 1, cnt) + + pool.Close() + stat := pool.Statistics() + assert.Equal(t, uint64(0), stat.ReturnCnt) + assert.Equal(t, uint64(0), stat.CloseCnt) + + conn.Close() + stat = pool.Statistics() + assert.Equal(t, uint64(1), stat.ReturnCnt) + assert.Equal(t, uint64(1), stat.CloseCnt) + + echoServer.Close() +}