You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pegasus.apache.org by wa...@apache.org on 2023/05/25 10:55:20 UTC

[incubator-pegasus] 21/28: feat(collector): migrate the collector from pegasus-kv/collector (#1461)

This is an automated email from the ASF dual-hosted git repository.

wangdan pushed a commit to branch migrate-metrics-dev
in repository https://gitbox.apache.org/repos/asf/incubator-pegasus.git

commit 1440a611cba27a9f7875dbac6fa0c9e1247260c9
Author: Yingchun Lai <la...@apache.org>
AuthorDate: Thu Apr 27 09:47:02 2023 +0800

    feat(collector): migrate the collector from pegasus-kv/collector (#1461)
    
    Co-authored-by: neverchanje <wu...@xiaomi.com>
---
 .github/workflows/lint_and_test_collector.yml      |  72 ++
 .github/workflows/module_labeler_conf.yml          |   2 +
 .github/workflows/standardization_lint.yaml        |  19 +-
 .gitignore                                         |   3 +
 .../module_labeler_conf.yml => collector/Makefile  |  42 +-
 collector/README.md                                |  28 +
 collector/aggregate/aggregatable.go                |  92 +++
 collector/aggregate/aggregator.go                  | 173 +++++
 collector/aggregate/aggregator_test.go             |  97 +++
 collector/aggregate/history.go                     | 111 ++++
 collector/aggregate/history_test.go                |  40 ++
 collector/aggregate/hook.go                        |  69 ++
 collector/aggregate/hook_test.go                   |  51 ++
 collector/aggregate/perf_client.go                 | 237 +++++++
 collector/aggregate/perf_client_test.go            |  49 ++
 collector/aggregate/perf_counter_decoder.go        |  62 ++
 collector/aggregate/perf_counter_decoder_test.go   |  61 ++
 collector/aggregate/perf_session.go                |  91 +++
 collector/aggregate/table_stats.go                 | 134 ++++
 collector/avail/detector.go                        | 124 ++++
 .../config.yml                                     |  65 +-
 collector/go.mod                                   |  47 ++
 collector/go.sum                                   | 739 +++++++++++++++++++++
 collector/hotspot/algo.go                          |  18 +
 collector/main.go                                  |  99 +++
 collector/metrics/falcon_sink.go                   | 144 ++++
 collector/metrics/prometheus_sink.go               | 110 +++
 collector/metrics/sink.go                          |  53 ++
 collector/templates/index.html                     |  80 +++
 collector/usage/usage_recorder.go                  | 115 ++++
 collector/webui/index.go                           |  84 +++
 collector/webui/tables.go                          |  47 ++
 collector/webui/webserver.go                       |  61 ++
 33 files changed, 3139 insertions(+), 80 deletions(-)

diff --git a/.github/workflows/lint_and_test_collector.yml b/.github/workflows/lint_and_test_collector.yml
new file mode 100644
index 000000000..63909e0ae
--- /dev/null
+++ b/.github/workflows/lint_and_test_collector.yml
@@ -0,0 +1,72 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+---
+# workflow name
+name: Golang Lint and Unit Test - collector
+
+# on events
+on:
+  # run on each pull request
+  pull_request:
+    types: [ synchronize, reopened, opened ]
+    branches:
+      - master
+      - 'v[0-9]+.*' # release branch
+      - ci-test # testing branch for github action
+      - '*dev'
+    paths:
+      - collector/**
+
+  # for manually triggering workflow
+  workflow_dispatch:
+
+# workflow tasks
+jobs:
+  lint:
+    name: Lint
+    runs-on: ubuntu-20.04
+    steps:
+      - name: Checkout
+        uses: actions/checkout@v3
+        with:
+          fetch-depth: 1
+      - name: Set up Go
+        uses: actions/setup-go@v4
+        with:
+          go-version: 1.14
+          cache: false
+      - name: Lint
+        uses: golangci/golangci-lint-action@v3
+        with:
+          version: v1.29
+          working-directory: ./collector
+
+  build:
+    name: Build
+    runs-on: ubuntu-20.04
+    steps:
+      - name: Checkout
+        uses: actions/checkout@v3
+        with:
+          fetch-depth: 1
+      - name: Set up Go
+        uses: actions/setup-go@v2
+        with:
+          go-version: 1.14
+      - name: Build
+        working-directory: ./collector
+        run: make
diff --git a/.github/workflows/module_labeler_conf.yml b/.github/workflows/module_labeler_conf.yml
index d5648edf9..ad23e4bf0 100644
--- a/.github/workflows/module_labeler_conf.yml
+++ b/.github/workflows/module_labeler_conf.yml
@@ -19,6 +19,8 @@ github:
   - .github/**/*
 admin-cli:
   - admin-cli/**/*
+collector:
+  - collector/**/*
 docker:
   - docker/**/*
 go-client:
diff --git a/.github/workflows/standardization_lint.yaml b/.github/workflows/standardization_lint.yaml
index ef94b8995..99a4ad836 100644
--- a/.github/workflows/standardization_lint.yaml
+++ b/.github/workflows/standardization_lint.yaml
@@ -62,15 +62,16 @@ jobs:
           bodyRegex: '#(\d+)'
           bodyURLRegex: 'http(s?):\/\/(github.com)(\/apache)(\/incubator-pegasus)(\/issues)\/\d+'
 
-  dockerfile_linter:
-    name: Lint Dockerfile
-    runs-on: ubuntu-latest
-    steps:
-      - uses: actions/checkout@v3
-      - uses: hadolint/hadolint-action@v3.1.0
-        with:
-          recursive: true
-          ignore: 'DL3033,DL3013,DL3059,SC2086,DL3003,SC2164,DL3008,DL3007,DL3006,DL4001'
+# TODO(yingchun): hadolint/hadolint-action@v3.1.0 is not allowed to be used in apache/incubator-pegasus.
+#  dockerfile_linter:
+#    name: Lint Dockerfile
+#    runs-on: ubuntu-latest
+#    steps:
+#      - uses: actions/checkout@v3
+#      - uses: hadolint/hadolint-action@v3.1.0
+#        with:
+#          recursive: true
+#          ignore: 'DL3033,DL3013,DL3059,SC2086,DL3003,SC2164,DL3008,DL3007,DL3006,DL4001'
 
   license_check:
     name: Check License
diff --git a/.gitignore b/.gitignore
index 457dbf309..a3ce7122c 100644
--- a/.gitignore
+++ b/.gitignore
@@ -271,3 +271,6 @@ thirdparty/output/
 
 #macOS
 .DS_Store
+
+#collector
+collector/collector
diff --git a/.github/workflows/module_labeler_conf.yml b/collector/Makefile
similarity index 58%
copy from .github/workflows/module_labeler_conf.yml
copy to collector/Makefile
index d5648edf9..7fab99e04 100644
--- a/.github/workflows/module_labeler_conf.yml
+++ b/collector/Makefile
@@ -14,39 +14,11 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
----
-github:
-  - .github/**/*
-admin-cli:
-  - admin-cli/**/*
-docker:
-  - docker/**/*
-go-client:
-  - go-client/**/*
-java-client:
-  - java-client/**/*
-nodejs-client:
-  - nodejs-client/**/*
-pegic:
-  - pegic/**/*
-python-client:
-  - python-client/**/*
-scala-client:
-  - scala-client/**/*
-thirdparty:
-  - thirdparty/**/*
-thrift:
-  - '**/*.thrift'
-docs:
-  - '**/*.md'
-scripts:
-  - 'scripts/**/*'
-  - '**/*.sh'
+
 build:
-  - 'cmake_modules/**/*'
-  - '**/CMakeLists.txt'
-  - 'run.sh'
-cpp:
-  # TODO(yingchun): add more fine-grained labels
-  - 'src/**/*.h'
-  - 'src/**/*.cpp'
+	go mod tidy
+	go mod verify
+	go build -o collector
+
+fmt:
+	go fmt ./...
diff --git a/collector/README.md b/collector/README.md
new file mode 100644
index 000000000..7f8b09ac2
--- /dev/null
+++ b/collector/README.md
@@ -0,0 +1,28 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+-->
+
+# Pegasus Collector
+
+[中文文档]
+
+Collector is a part of the Pegasus ecosystem that serves as:
+
+1. the service availability detector
+2. the hotkey detector
+3. the capacity units recorder
diff --git a/collector/aggregate/aggregatable.go b/collector/aggregate/aggregatable.go
new file mode 100644
index 000000000..7ff8c5fb3
--- /dev/null
+++ b/collector/aggregate/aggregatable.go
@@ -0,0 +1,92 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+var v1Tov2MetricsConversion = map[string]string{
+	"replica*app.pegasus*get_qps":                                  "get_qps",
+	"replica*app.pegasus*multi_get_qps":                            "multi_get_qps",
+	"replica*app.pegasus*put_qps":                                  "put_qps",
+	"replica*app.pegasus*multi_put_qps":                            "multi_put_qps",
+	"replica*app.pegasus*remove_qps":                               "remove_qps",
+	"replica*app.pegasus*multi_remove_qps":                         "multi_remove_qps",
+	"replica*app.pegasus*incr_qps":                                 "incr_qps",
+	"replica*app.pegasus*check_and_set_qps":                        "check_and_set_qps",
+	"replica*app.pegasus*check_and_mutate_qps":                     "check_and_mutate_qps",
+	"replica*app.pegasus*scan_qps":                                 "scan_qps",
+	"replica*eon.replica*backup_request_qps":                       "backup_request_qps",
+	"replica*app.pegasus*duplicate_qps":                            "duplicate_qps",
+	"replica*app.pegasus*dup_shipped_ops":                          "dup_shipped_ops",
+	"replica*app.pegasus*dup_failed_shipping_ops":                  "dup_failed_shipping_ops",
+	"replica*app.pegasus*get_bytes":                                "get_bytes",
+	"replica*app.pegasus*multi_get_bytes":                          "multi_get_bytes",
+	"replica*app.pegasus*scan_bytes":                               "scan_bytes",
+	"replica*app.pegasus*put_bytes":                                "put_bytes",
+	"replica*app.pegasus*multi_put_bytes":                          "multi_put_bytes",
+	"replica*app.pegasus*check_and_set_bytes":                      "check_and_set_bytes",
+	"replica*app.pegasus*check_and_mutate_bytes":                   "check_and_mutate_bytes",
+	"replica*app.pegasus*recent.read.cu":                           "recent_read_cu",
+	"replica*app.pegasus*recent.write.cu":                          "recent_write_cu",
+	"replica*app.pegasus*recent.expire.count":                      "recent_expire_count",
+	"replica*app.pegasus*recent.filter.count":                      "recent_filter_count",
+	"replica*app.pegasus*recent.abnormal.count":                    "recent_abnormal_count",
+	"replica*eon.replica*recent.write.throttling.delay.count":      "recent_write_throttling_delay_count",
+	"replica*eon.replica*recent.write.throttling.reject.count":     "recent_write_throttling_reject_count",
+	"replica*app.pegasus*disk.storage.sst(MB)":                     "sst_storage_mb",
+	"replica*app.pegasus*disk.storage.sst.count":                   "sst_count",
+	"replica*app.pegasus*rdb.block_cache.hit_count":                "rdb_block_cache_hit_count",
+	"replica*app.pegasus*rdb.block_cache.total_count":              "rdb_block_cache_total_count",
+	"replica*app.pegasus*rdb.index_and_filter_blocks.memory_usage": "rdb_index_and_filter_blocks_mem_usage",
+	"replica*app.pegasus*rdb.memtable.memory_usage":                "rdb_memtable_mem_usage",
+	"replica*app.pegasus*rdb.estimate_num_keys":                    "rdb_estimate_num_keys",
+	"replica*app.pegasus*rdb.bf_seek_negatives":                    "rdb_bf_seek_negatives",
+	"replica*app.pegasus*rdb.bf_seek_total":                        "rdb_bf_seek_total",
+	"replica*app.pegasus*rdb.bf_point_positive_true":               "rdb_bf_point_positive_true",
+	"replica*app.pegasus*rdb.bf_point_positive_total":              "rdb_bf_point_positive_total",
+	"replica*app.pegasus*rdb.bf_point_negatives":                   "rdb_bf_point_negatives",
+}
+
+var aggregatableSet = map[string]interface{}{
+	"read_qps":    nil,
+	"write_qps":   nil,
+	"read_bytes":  nil,
+	"write_bytes": nil,
+}
+
+// aggregatable returns whether the counter is to be aggregated on collector,
+// including v1Tov2MetricsConversion and aggregatableSet.
+func aggregatable(pc *partitionPerfCounter) bool {
+	v2Name, found := v1Tov2MetricsConversion[pc.name]
+	if found { // ignored
+		pc.name = v2Name
+		return true // listed above are all aggregatable
+	}
+	_, found = aggregatableSet[pc.name]
+	return found
+}
+
+// AllMetrics returns metrics tracked within this collector.
+// The sets of metrics from cluster level and table level are completely equal.
+func AllMetrics() (res []string) {
+	for _, newName := range v1Tov2MetricsConversion {
+		res = append(res, newName)
+	}
+	for name := range aggregatableSet {
+		res = append(res, name)
+	}
+	return res
+}
diff --git a/collector/aggregate/aggregator.go b/collector/aggregate/aggregator.go
new file mode 100644
index 000000000..7e0cf50b2
--- /dev/null
+++ b/collector/aggregate/aggregator.go
@@ -0,0 +1,173 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+	"gopkg.in/tomb.v2"
+)
+
+// TableStatsAggregator aggregates the metric on each partition into table-level metrics.
+// It's reponsible for all tables in the pegasus cluster.
+// After all TableStats have been collected, TableStatsAggregator sums them up into a
+// ClusterStats. Users of this pacakage can use the hooks to watch every changes of the stats.
+type TableStatsAggregator interface {
+	Aggregate() (map[int32]*TableStats, *ClusterStats, error)
+
+	Close()
+}
+
+// NewTableStatsAggregator returns a TableStatsAggregator instance.
+func NewTableStatsAggregator(metaAddrs []string) TableStatsAggregator {
+	return &tableStatsAggregator{
+		tables: make(map[int32]*TableStats),
+		client: NewPerfClient(metaAddrs),
+	}
+}
+
+type tableStatsAggregator struct {
+	tables   map[int32]*TableStats
+	allStats *ClusterStats
+
+	client *PerfClient
+}
+
+// Start looping for metrics aggregation
+func Start(tom *tomb.Tomb) {
+	aggregateInterval := viper.GetDuration("metrics.report_interval")
+	ticker := time.NewTicker(aggregateInterval)
+
+	metaAddr := viper.GetString("meta_server")
+	iAg := NewTableStatsAggregator([]string{metaAddr})
+	ag := iAg.(*tableStatsAggregator)
+
+	for {
+		select {
+		case <-tom.Dying(): // check if context cancelled
+			return
+		case <-ticker.C:
+		}
+
+		_, _, err := ag.Aggregate()
+		if err != nil {
+			log.Error(err)
+		}
+
+		// produce stats for the hooks
+		var batchTableStats []TableStats
+		for _, table := range ag.tables {
+			batchTableStats = append(batchTableStats, *table)
+		}
+		ag.aggregateClusterStats()
+		hooksManager.afterTableStatsEmitted(batchTableStats, *ag.allStats)
+	}
+}
+
+func (ag *tableStatsAggregator) Aggregate() (map[int32]*TableStats, *ClusterStats, error) {
+	err := ag.updateTableMap()
+	if err != nil {
+		return nil, nil, fmt.Errorf("failed to aggregate: %s", err)
+	}
+
+	// TODO(wutao1): reduce meta queries for listing nodes
+	partitions, err := ag.client.GetPartitionStats()
+	if err != nil {
+		return nil, nil, fmt.Errorf("failed to aggregate: %s", err)
+	}
+	for _, p := range partitions {
+		ag.updatePartitionStat(p)
+	}
+
+	for _, table := range ag.tables {
+		table.aggregate()
+	}
+
+	return ag.tables, ag.allStats, nil
+}
+
+func (ag *tableStatsAggregator) Close() {
+	ag.client.Close()
+}
+
+func (ag *tableStatsAggregator) aggregateClusterStats() {
+	ag.allStats = &ClusterStats{
+		Stats:     make(map[string]float64),
+		Timestamp: time.Now(),
+	}
+	for _, table := range ag.tables {
+		for k, v := range table.Stats {
+			ag.allStats.Stats[k] += v
+		}
+	}
+}
+
+// Some tables may disappear (be dropped) or first show up.
+// This function maintains the local table map
+// to keep consistent with the pegasus cluster.
+func (ag *tableStatsAggregator) updateTableMap() error {
+	tables, err := ag.client.listTables()
+	if err != nil {
+		return err
+	}
+	ag.doUpdateTableMap(tables)
+	return nil
+}
+
+func (ag *tableStatsAggregator) doUpdateTableMap(tables []*admin.AppInfo) {
+	currentTableSet := make(map[int32]*struct{})
+	for _, tb := range tables {
+		currentTableSet[tb.AppID] = nil
+		if _, found := ag.tables[tb.AppID]; !found {
+			// non-exisistent table, create it
+			ag.tables[tb.AppID] = newTableStats(tb)
+			log.Infof("found new table: %+v", tb)
+
+			// TODO(wutao1): some tables may have partitions splitted,
+			//               recreate the tableStats then.
+		}
+	}
+	for appID, tb := range ag.tables {
+		// disappeared table, delete it
+		if _, found := currentTableSet[appID]; !found {
+			log.Infof("remove table from collector: {AppID: %d, PartitionCount: %d}", appID, len(tb.Partitions))
+			delete(ag.tables, appID)
+
+			hooksManager.afterTableDropped(appID)
+		}
+	}
+}
+
+// Update the counter value.
+func (ag *tableStatsAggregator) updatePartitionStat(pc *PartitionStats) {
+	tb, found := ag.tables[pc.Gpid.Appid]
+	if !found {
+		// Ignore the perf-counter because there's currently no such table
+		return
+	}
+	part, found := tb.Partitions[int(pc.Gpid.PartitionIndex)]
+	if !found {
+		log.Errorf("no such partition %+v", pc.Gpid)
+		return
+	}
+	*part = *pc
+}
diff --git a/collector/aggregate/aggregator_test.go b/collector/aggregate/aggregator_test.go
new file mode 100644
index 000000000..4b73e9213
--- /dev/null
+++ b/collector/aggregate/aggregator_test.go
@@ -0,0 +1,97 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestUpdateLocalTableMap(t *testing.T) {
+	ag := &tableStatsAggregator{
+		client: NewPerfClient([]string{"127.0.0.1:34601"}),
+		tables: make(map[int32]*TableStats),
+	}
+	err := ag.updateTableMap()
+	assert.Nil(t, err)
+	assert.Equal(t, len(ag.tables), 2)
+	assert.Equal(t, len(ag.tables[1].Partitions), 4) // test
+	assert.Equal(t, len(ag.tables[2].Partitions), 8) // stat
+
+	tables := []*admin.AppInfo{
+		{AppID: 1, AppName: "stat", PartitionCount: 4},
+		{AppID: 2, AppName: "test", PartitionCount: 8},
+		{AppID: 3, AppName: "new_table", PartitionCount: 16},
+	}
+	ag.doUpdateTableMap(tables)
+	assert.Equal(t, len(ag.tables), 3)
+	assert.Equal(t, len(ag.tables[3].Partitions), 16)
+
+	tables = []*admin.AppInfo{
+		{AppID: 1, AppName: "stat", PartitionCount: 4},
+	}
+	ag.doUpdateTableMap(tables)
+	assert.Equal(t, len(ag.tables), 1)
+	assert.Equal(t, len(ag.tables[1].Partitions), 4)
+}
+
+func TestUpdatePartitionStats(t *testing.T) {
+	ag := &tableStatsAggregator{
+		tables: make(map[int32]*TableStats),
+	}
+	tables := []*admin.AppInfo{
+		{AppID: 1, AppName: "stat", PartitionCount: 4},
+	}
+	ag.doUpdateTableMap(tables)
+
+	pc := decodePartitionPerfCounter("replica*app.pegasus*recent.abnormal.count@1.2", 100)
+	assert.NotNil(t, pc)
+
+	ag.updatePartitionStat(&PartitionStats{
+		Gpid: base.Gpid{Appid: 1, PartitionIndex: 2},
+		Addr: "127.0.0.1:34601",
+		Stats: map[string]float64{
+			"replica*app.pegasus*recent.abnormal.count": 100,
+		},
+	})
+
+	part := ag.tables[1].Partitions[2]
+	assert.Contains(t, part.Stats, pc.name)
+	assert.Equal(t, part.Stats[pc.name], float64(100))
+	assert.Equal(t, part.Addr, "127.0.0.1:34601")
+}
+
+func TestAggregate(t *testing.T) {
+	ag := NewTableStatsAggregator([]string{"127.0.0.1:34601"})
+	tableStats, allStat, err := ag.Aggregate()
+	assert.Nil(t, err)
+	assert.Greater(t, len(allStat.Stats), 0)
+
+	assert.Equal(t, len(tableStats), 2)
+
+	// ensure len(tableStats) == len(partitionStats) == len(clusterStats)
+	for _, tb := range tableStats {
+		assert.Equal(t, len(tb.Stats), len(allStat.Stats))
+		for _, p := range tb.Partitions {
+			assert.Equal(t, len(p.Stats), len(allStat.Stats))
+		}
+	}
+}
diff --git a/collector/aggregate/history.go b/collector/aggregate/history.go
new file mode 100644
index 000000000..27e6d10ee
--- /dev/null
+++ b/collector/aggregate/history.go
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"container/list"
+	"sync"
+)
+
+const (
+	historyMaxCapacity = 10
+)
+
+// threadSafeHistory is a time-ordered queue of stats.
+type threadSafeHistory struct {
+	lock sync.RWMutex
+
+	stats    *list.List
+	capacity int
+}
+
+// Emit a TableStats to the history. Will remove the oldest record from history.
+func (h *threadSafeHistory) emit(stat interface{}) {
+	h.lock.Lock()
+	defer h.lock.Unlock()
+
+	if h.stats.Len() == h.capacity {
+		h.stats.Remove(h.stats.Front())
+	}
+	h.stats.PushBack(stat)
+}
+
+func newHistory(capacity int) *threadSafeHistory {
+	return &threadSafeHistory{
+		stats:    list.New(),
+		capacity: capacity,
+	}
+}
+
+type historyStore struct {
+	lock sync.RWMutex
+
+	tables  map[int]*threadSafeHistory
+	cluster *threadSafeHistory
+}
+
+var globalHistoryStore = &historyStore{
+	tables:  make(map[int]*threadSafeHistory),
+	cluster: newHistory(historyMaxCapacity),
+}
+
+// SnapshotClusterStats takes a snapshot from the history. The returned array is ordered by time.
+func SnapshotClusterStats() []ClusterStats {
+	s := globalHistoryStore
+
+	s.lock.RLock()
+	defer s.lock.RUnlock()
+
+	var result []ClusterStats
+	l := s.cluster.stats
+	for e := l.Front(); e != nil; e = e.Next() {
+		stat, _ := e.Value.(*ClusterStats)
+		result = append(result, *stat)
+	}
+	return result
+}
+
+func init() {
+	initHistoryStore()
+}
+
+func initHistoryStore() {
+	AddHookAfterTableStatEmitted(func(stats []TableStats, allStat ClusterStats) {
+		s := globalHistoryStore
+
+		s.lock.Lock()
+		defer s.lock.Unlock()
+		for _, stat := range stats {
+			history, found := s.tables[stat.AppID]
+			if !found {
+				history = newHistory(historyMaxCapacity)
+				s.tables[stat.AppID] = history
+			}
+			history.emit(&stat)
+		}
+		s.cluster.emit(&allStat)
+	})
+
+	AddHookAfterTableDropped(func(appID int) {
+		s := globalHistoryStore
+
+		s.lock.Lock()
+		defer s.lock.Unlock()
+		delete(s.tables, appID)
+	})
+}
diff --git a/collector/aggregate/history_test.go b/collector/aggregate/history_test.go
new file mode 100644
index 000000000..e4bbbfa06
--- /dev/null
+++ b/collector/aggregate/history_test.go
@@ -0,0 +1,40 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestHistory(t *testing.T) {
+	hooksManager = tableStatsHooksManager{}
+	initHistoryStore()
+
+	for i := 0; i < historyMaxCapacity*2; i++ {
+		hooksManager.afterTableStatsEmitted([]TableStats{},
+			ClusterStats{Stats: map[string]float64{"write": 100.0 * float64(i)}, Timestamp: time.Now()})
+	}
+	clusterStats := SnapshotClusterStats()
+	assert.Equal(t, len(clusterStats), historyMaxCapacity)
+	for i := 0; i < historyMaxCapacity; i++ {
+		assert.Equal(t, clusterStats[i].Stats["write"], float64(historyMaxCapacity+i)*100.0)
+	}
+}
diff --git a/collector/aggregate/hook.go b/collector/aggregate/hook.go
new file mode 100644
index 000000000..e20af348a
--- /dev/null
+++ b/collector/aggregate/hook.go
@@ -0,0 +1,69 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import "sync"
+
+// HookAfterTableStatEmitted is a hook of event that new TableStats are generated.
+// Each call of the hook handles a batch of tables.
+type HookAfterTableStatEmitted func(stats []TableStats, allStats ClusterStats)
+
+// AddHookAfterTableStatEmitted adds a hook of event that a new TableStats is generated.
+func AddHookAfterTableStatEmitted(hk HookAfterTableStatEmitted) {
+	m := &hooksManager
+	m.lock.Lock()
+	defer m.lock.Unlock()
+	m.emittedHooks = append(m.emittedHooks, hk)
+}
+
+// HookAfterTableDropped is a hook of event that a table is dropped.
+type HookAfterTableDropped func(appID int)
+
+// AddHookAfterTableDropped adds a hook of event that a table is dropped.
+func AddHookAfterTableDropped(hk HookAfterTableDropped) {
+	m := &hooksManager
+	m.lock.Lock()
+	defer m.lock.Unlock()
+	m.droppedHooks = append(m.droppedHooks, hk)
+}
+
+type tableStatsHooksManager struct {
+	lock         sync.RWMutex
+	emittedHooks []HookAfterTableStatEmitted
+	droppedHooks []HookAfterTableDropped
+}
+
+func (m *tableStatsHooksManager) afterTableStatsEmitted(stats []TableStats, allStat ClusterStats) {
+	m.lock.RLock()
+	defer m.lock.RUnlock()
+
+	for _, hook := range m.emittedHooks {
+		hook(stats, allStat)
+	}
+}
+
+func (m *tableStatsHooksManager) afterTableDropped(appID int32) {
+	m.lock.RLock()
+	defer m.lock.RUnlock()
+
+	for _, hook := range m.droppedHooks {
+		hook(int(appID))
+	}
+}
+
+var hooksManager tableStatsHooksManager
diff --git a/collector/aggregate/hook_test.go b/collector/aggregate/hook_test.go
new file mode 100644
index 000000000..a1a853314
--- /dev/null
+++ b/collector/aggregate/hook_test.go
@@ -0,0 +1,51 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestAddHook(t *testing.T) {
+	var actualTableStats []TableStats
+	var actualClusterStats ClusterStats
+	AddHookAfterTableStatEmitted(func(stats []TableStats, allStats ClusterStats) {
+		actualTableStats = stats
+		actualClusterStats = allStats
+	})
+
+	t1 := TableStats{TableName: "test", Stats: map[string]float64{"write": 512.0}, Timestamp: time.Now()}
+	t2 := TableStats{TableName: "stat", Stats: map[string]float64{"write": 256.0}, Timestamp: time.Now()}
+	all := ClusterStats{Stats: map[string]float64{"write": 768.0}}
+
+	ch := make(chan interface{})
+	go func() {
+		hooksManager.afterTableStatsEmitted([]TableStats{t1, t2}, all)
+		ch <- nil
+	}()
+	<-ch
+
+	assert.EqualValues(t, actualTableStats, []TableStats{t1, t2})
+	assert.EqualValues(t, actualClusterStats, all)
+
+	// clear up
+	hooksManager = tableStatsHooksManager{}
+}
diff --git a/collector/aggregate/perf_client.go b/collector/aggregate/perf_client.go
new file mode 100644
index 000000000..26739c116
--- /dev/null
+++ b/collector/aggregate/perf_client.go
@@ -0,0 +1,237 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"context"
+	"fmt"
+	"sync"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/apache/incubator-pegasus/go-client/session"
+	log "github.com/sirupsen/logrus"
+	batchErr "k8s.io/apimachinery/pkg/util/errors"
+)
+
+// PerfClient manages sessions to all replica nodes.
+type PerfClient struct {
+	meta *session.MetaManager
+
+	nodes map[string]*PerfSession
+}
+
+// GetPartitionStats retrieves all the partition stats from replica nodes.
+// NOTE: Only the primaries are counted.
+func (m *PerfClient) GetPartitionStats() ([]*PartitionStats, error) {
+	m.updateNodes()
+
+	partitions, err := m.preparePrimariesStats()
+	if err != nil {
+		return nil, err
+	}
+
+	nodeStats, err := m.GetNodeStats("@")
+	if err != nil {
+		return nil, err
+	}
+
+	for _, n := range nodeStats {
+		for name, value := range n.Stats {
+			perfCounter := decodePartitionPerfCounter(name, value)
+			if perfCounter == nil {
+				continue
+			}
+			if !aggregatable(perfCounter) {
+				continue
+			}
+			part := partitions[perfCounter.gpid]
+			if part == nil || part.Addr != n.Addr {
+				// if this node is not the primary of this partition
+				continue
+			}
+
+			part.Stats[perfCounter.name] = perfCounter.value
+		}
+	}
+
+	var ret []*PartitionStats
+	for _, part := range partitions {
+		extendStats(&part.Stats)
+		ret = append(ret, part)
+	}
+	return ret, nil
+}
+
+// getPrimaries returns mapping of [partition -> primary address]
+func (m *PerfClient) getPrimaries() (map[base.Gpid]string, error) {
+	tables, err := m.listTables()
+	if err != nil {
+		return nil, err
+	}
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
+	defer cancel()
+
+	result := make(map[base.Gpid]string)
+	var mu sync.Mutex
+	var funcs []func() error
+
+	for _, table := range tables {
+		tb := table
+		funcs = append(funcs, func() (subErr error) {
+			tableCfg, err := m.meta.QueryConfig(ctx, tb.AppName)
+			if err != nil {
+				return fmt.Errorf("failed on table(%s): %s", tb.AppName, err)
+			}
+			mu.Lock()
+			for _, p := range tableCfg.Partitions {
+				result[*p.Pid] = p.Primary.GetAddress()
+			}
+			mu.Unlock()
+			return nil
+		})
+	}
+	return result, batchErr.AggregateGoroutines(funcs...)
+}
+
+func (m *PerfClient) preparePrimariesStats() (map[base.Gpid]*PartitionStats, error) {
+	primaries, err := m.getPrimaries()
+	if err != nil {
+		return nil, err
+	}
+	partitions := make(map[base.Gpid]*PartitionStats)
+	for p, addr := range primaries {
+		partitions[p] = &PartitionStats{
+			Gpid:  p,
+			Stats: make(map[string]float64),
+			Addr:  addr,
+		}
+	}
+	return partitions, nil
+}
+
+// NodeStat contains the stats of a replica node.
+type NodeStat struct {
+	// Address of the replica node.
+	Addr string
+
+	// perfCounter's name -> the value.
+	Stats map[string]float64
+}
+
+// GetNodeStats retrieves all the stats matched with `filter` from replica nodes.
+func (m *PerfClient) GetNodeStats(filter string) ([]*NodeStat, error) {
+	m.updateNodes()
+
+	// concurrently send RPC for perf-counters.
+	var results []*NodeStat
+	var funcs []func() error
+	var mu sync.Mutex
+
+	for _, node := range m.nodes {
+		n := node
+		funcs = append(funcs, func() (subErr error) {
+			stat := &NodeStat{
+				Addr:  n.Address,
+				Stats: make(map[string]float64),
+			}
+			perfCounters, err := n.GetPerfCounters(filter)
+			if err != nil {
+				return fmt.Errorf("failed on node(%s): %s", n.Address, err)
+			}
+			for _, p := range perfCounters {
+				stat.Stats[p.Name] = p.Value
+			}
+
+			mu.Lock()
+			results = append(results, stat)
+			defer mu.Unlock()
+			return nil
+		})
+	}
+	return results, batchErr.AggregateGoroutines(funcs...)
+}
+
+func (m *PerfClient) listNodes() ([]*admin.NodeInfo, error) {
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
+	defer cancel()
+	resp, err := m.meta.ListNodes(ctx, &admin.ListNodesRequest{
+		Status: admin.NodeStatus_NS_ALIVE,
+	})
+	if err != nil {
+		return nil, err
+	}
+	return resp.Infos, nil
+}
+
+func (m *PerfClient) listTables() ([]*admin.AppInfo, error) {
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
+	defer cancel()
+	resp, err := m.meta.ListApps(ctx, &admin.ListAppsRequest{
+		Status: admin.AppStatus_AS_AVAILABLE,
+	})
+	if err != nil {
+		return nil, err
+	}
+	return resp.Infos, nil
+}
+
+// updateNodes
+func (m *PerfClient) updateNodes() {
+	nodeInfos, err := m.listNodes()
+	if err != nil {
+		log.Error("skip updating nodes due to list-nodes RPC failure: ", err)
+		return
+	}
+
+	newNodes := make(map[string]*PerfSession)
+	for _, n := range nodeInfos {
+		addr := n.Address.GetAddress()
+		node, found := m.nodes[addr]
+		if !found {
+			newNodes[addr] = NewPerfSession(addr)
+		} else {
+			newNodes[addr] = node
+		}
+	}
+	for n, client := range m.nodes {
+		// close the unused connections
+		if _, found := newNodes[n]; !found {
+			client.Close()
+		}
+	}
+	m.nodes = newNodes
+}
+
+// Close release the resources.
+func (m *PerfClient) Close() {
+	m.meta.Close()
+
+	for _, n := range m.nodes {
+		n.Close()
+	}
+}
+
+// NewPerfClient returns an instance of PerfClient.
+func NewPerfClient(metaAddrs []string) *PerfClient {
+	return &PerfClient{
+		meta:  session.NewMetaManager(metaAddrs, session.NewNodeSession),
+		nodes: make(map[string]*PerfSession),
+	}
+}
diff --git a/collector/aggregate/perf_client_test.go b/collector/aggregate/perf_client_test.go
new file mode 100644
index 000000000..264eb60b3
--- /dev/null
+++ b/collector/aggregate/perf_client_test.go
@@ -0,0 +1,49 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestPerfClientGetNodeStats(t *testing.T) {
+	pclient := NewPerfClient([]string{"127.0.0.1:34601"})
+	nodes, err := pclient.GetNodeStats("@")
+	assert.Nil(t, err)
+	assert.Greater(t, len(nodes), 0)
+	assert.Greater(t, len(nodes[0].Stats), 0)
+	for _, n := range nodes {
+		assert.NotEmpty(t, n.Addr)
+	}
+}
+
+func TestPerfClientGetPartitionStats(t *testing.T) {
+	pclient := NewPerfClient([]string{"127.0.0.1:34601"})
+	partitions, err := pclient.GetPartitionStats()
+	assert.Nil(t, err)
+	assert.Greater(t, len(partitions), 0)
+	assert.Greater(t, len(partitions[0].Stats), 0)
+	for _, p := range partitions {
+		assert.NotEmpty(t, p.Addr)
+		assert.NotEqual(t, p.Gpid, base.Gpid{Appid: 0, PartitionIndex: 0})
+		assert.NotEmpty(t, p.Stats)
+	}
+}
diff --git a/collector/aggregate/perf_counter_decoder.go b/collector/aggregate/perf_counter_decoder.go
new file mode 100644
index 000000000..f89e31840
--- /dev/null
+++ b/collector/aggregate/perf_counter_decoder.go
@@ -0,0 +1,62 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"strconv"
+	"strings"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+)
+
+type partitionPerfCounter struct {
+	name  string
+	gpid  base.Gpid
+	value float64
+}
+
+// decodePartitionPerfCounter implements the v1 version of metric decoding.
+func decodePartitionPerfCounter(name string, value float64) *partitionPerfCounter {
+	idx := strings.LastIndex(name, "@")
+	gpidStr := name[idx+1:]
+	appIDAndPartitionID := strings.Split(gpidStr, ".")
+	if len(appIDAndPartitionID) != 2 {
+		// special case: in some mis-desgined metrics, what follows after a '@' may not be a replica id
+		return nil
+	}
+	appIDAndPartitionID = appIDAndPartitionID[:2] // "AppID.PartitionIndex"
+	appID, err := strconv.Atoi(appIDAndPartitionID[0])
+	if err != nil {
+		return nil
+	}
+	partitionIndex, err := strconv.Atoi(appIDAndPartitionID[1])
+	if err != nil {
+		return nil
+	}
+	return &partitionPerfCounter{
+		name: name[:idx], // strip out the replica id
+		gpid: base.Gpid{
+			Appid:          int32(appID),
+			PartitionIndex: int32(partitionIndex),
+		},
+		value: value,
+	}
+}
+
+// TODO(wutao1): implement the v2 version of metric decoding according to
+// https://github.com/apache/incubator-pegasus/blob/master/rfcs/2020-08-27-metric-api.md
diff --git a/collector/aggregate/perf_counter_decoder_test.go b/collector/aggregate/perf_counter_decoder_test.go
new file mode 100644
index 000000000..e910cd9f8
--- /dev/null
+++ b/collector/aggregate/perf_counter_decoder_test.go
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestDecodePartitionPerfCounter(t *testing.T) {
+	tests := []struct {
+		name string
+
+		isNil          bool
+		counterName    string
+		appID          int32
+		partitionIndex int32
+	}{
+		{name: "replica*app.pegasus*get_latency@2.5.p999", isNil: true},
+
+		// server-level counter, does not contain gpid.
+		{name: "replica*eon.replica*table.level.RPC_RRDB_RRDB_CHECK_AND_MUTATE.latency(ns)@temp", isNil: true},
+		{
+			name:  "replica*eon.replica*table.level.RPC_RRDB_RRDB_MULTI_PUT.latency(ns)@temp.p999",
+			isNil: true,
+		},
+
+		{
+			name:           "replica*app.pegasus*recent.abnormal.count@1.2",
+			counterName:    "replica*app.pegasus*recent.abnormal.count",
+			appID:          1,
+			partitionIndex: 2,
+		},
+	}
+
+	for _, tt := range tests {
+		pc := decodePartitionPerfCounter(tt.name, 1.0)
+		assert.Equal(t, pc == nil, tt.isNil, tt.name)
+		if pc != nil {
+			assert.Equal(t, pc.name, tt.counterName)
+			assert.Equal(t, pc.gpid, base.Gpid{Appid: tt.appID, PartitionIndex: tt.partitionIndex})
+		}
+	}
+}
diff --git a/collector/aggregate/perf_session.go b/collector/aggregate/perf_session.go
new file mode 100644
index 000000000..9e0665ac6
--- /dev/null
+++ b/collector/aggregate/perf_session.go
@@ -0,0 +1,91 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"context"
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/admin"
+	"github.com/apache/incubator-pegasus/go-client/session"
+	"github.com/tidwall/gjson"
+)
+
+// PerfSession is a client to get perf-counters from a Pegasus ReplicaServer.
+type PerfSession struct {
+	session.NodeSession
+
+	Address string
+}
+
+// PerfCounter is a Pegasus perf-counter.
+type PerfCounter struct {
+	Name  string
+	Value float64
+}
+
+func (p *PerfCounter) String() string {
+	return fmt.Sprintf("{Name: %s, Value: %f}", p.Name, p.Value)
+}
+
+// NewPerfSession returns an instance of PerfSession.
+func NewPerfSession(addr string) *PerfSession {
+	return &PerfSession{
+		Address:     addr,
+		NodeSession: session.NewNodeSession(addr, session.NodeTypeReplica),
+	}
+}
+
+// WrapPerf returns an instance of PerfSession using an existed session.
+func WrapPerf(addr string, session session.NodeSession) *PerfSession {
+	return &PerfSession{
+		Address:     addr,
+		NodeSession: session,
+	}
+}
+
+// GetPerfCounters retrieves all perf-counters matched with `filter` from the remote node.
+func (c *PerfSession) GetPerfCounters(filter string) ([]*PerfCounter, error) {
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
+	defer cancel()
+	rc := &admin.RemoteCommand{
+		Command:   "perf-counters-by-substr",
+		Arguments: []string{filter},
+	}
+
+	result, err := rc.Call(ctx, c.NodeSession)
+	if err != nil {
+		return nil, err
+	}
+	resultJSON := gjson.Parse(result)
+	perfCounters := resultJSON.Get("counters").Array()
+	var ret []*PerfCounter
+	for _, perfCounter := range perfCounters {
+		ret = append(ret, &PerfCounter{
+			Name:  perfCounter.Get("name").String(),
+			Value: perfCounter.Get("value").Float(),
+		})
+	}
+	return ret, nil
+}
+
+// Close terminates the session to replica.
+func (c *PerfSession) Close() {
+	c.NodeSession.Close()
+}
diff --git a/collector/aggregate/table_stats.go b/collector/aggregate/table_stats.go
new file mode 100644
index 000000000..62d9b7d10
--- /dev/null
+++ b/collector/aggregate/table_stats.go
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package aggregate
+
+import (
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+)
+
+// PartitionStats is a set of metrics retrieved from this partition.
+type PartitionStats struct {
+	Gpid base.Gpid
+
+	// Address of the primary where this partition locates.
+	Addr string
+
+	// perfCounter's name -> the value.
+	Stats map[string]float64
+}
+
+// TableStats has the aggregated metrics for this table.
+type TableStats struct {
+	TableName string
+	AppID     int
+
+	Partitions map[int]*PartitionStats
+
+	// the time when the stats was generated
+	Timestamp time.Time
+
+	// The aggregated value of table metrics.
+	// perfCounter's name -> the value.
+	Stats map[string]float64
+}
+
+// ClusterStats is the aggregated metrics for all the TableStats in this cluster.
+// For example, 3 tables with "write_qps" [25, 70, 100] are summed up to
+// `Stats: {"write_qps" : 195}`.
+type ClusterStats struct {
+	Timestamp time.Time
+
+	Stats map[string]float64
+}
+
+func newTableStats(info *admin.AppInfo) *TableStats {
+	tb := &TableStats{
+		TableName:  info.AppName,
+		AppID:      int(info.AppID),
+		Partitions: make(map[int]*PartitionStats),
+		Stats:      make(map[string]float64),
+		Timestamp:  time.Now(),
+	}
+	for i := 0; i < int(info.PartitionCount); i++ {
+		tb.Partitions[i] = &PartitionStats{
+			Gpid:  base.Gpid{Appid: int32(info.AppID), PartitionIndex: int32(i)},
+			Stats: make(map[string]float64),
+		}
+	}
+	return tb
+}
+
+func (tb *TableStats) aggregate() {
+	tb.Timestamp = time.Now()
+	for _, part := range tb.Partitions {
+		for name, value := range part.Stats {
+			tb.Stats[name] += value
+		}
+	}
+}
+
+func aggregateCustomStats(elements []string, stats *map[string]float64, resultName string) {
+	aggregated := float64(0)
+	for _, ele := range elements {
+		if v, found := (*stats)[ele]; found {
+			aggregated += v
+		}
+	}
+	(*stats)[resultName] = aggregated
+}
+
+// Extends the stat with read_qps/read_bytes/write_qps/write_bytes.
+func extendStats(stats *map[string]float64) {
+	var reads = []string{
+		"get",
+		"multi_get",
+		"scan",
+	}
+	var readQPS []string
+	for _, r := range reads {
+		readQPS = append(readQPS, r+"_qps")
+	}
+	var readBytes []string
+	for _, r := range reads {
+		readBytes = append(readBytes, r+"_bytes")
+	}
+	aggregateCustomStats(readQPS, stats, "read_qps")
+	aggregateCustomStats(readBytes, stats, "read_bytes")
+
+	var writes = []string{
+		"put",
+		"remove",
+		"multi_put",
+		"multi_remove",
+		"check_and_set",
+		"check_and_mutate",
+	}
+	var writeQPS []string
+	for _, w := range writes {
+		writeQPS = append(writeQPS, w+"_qps")
+	}
+	var writeBytes []string
+	for _, w := range writes {
+		writeBytes = append(writeBytes, w+"_bytes")
+	}
+	aggregateCustomStats(writeQPS, stats, "write_qps")
+	aggregateCustomStats(writeBytes, stats, "write_bytes")
+}
diff --git a/collector/avail/detector.go b/collector/avail/detector.go
new file mode 100644
index 000000000..1e23a6210
--- /dev/null
+++ b/collector/avail/detector.go
@@ -0,0 +1,124 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package avail
+
+import (
+	"context"
+	"sync/atomic"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/pegasus"
+	log "github.com/sirupsen/logrus"
+)
+
+// Detector periodically checks the service availability of the Pegasus cluster.
+type Detector interface {
+
+	// Start detection until the ctx cancelled. This method will block the current thread.
+	Start(ctx context.Context) error
+}
+
+// NewDetector returns a service-availability detector.
+func NewDetector(client pegasus.Client) Detector {
+	return &pegasusDetector{client: client}
+}
+
+type pegasusDetector struct {
+	// client reads and writes periodically to a specified table.
+	client      pegasus.Client
+	detectTable pegasus.TableConnector
+
+	detectInterval  time.Duration
+	detectTableName string
+
+	// timeout of a single detect
+	detectTimeout time.Duration
+
+	detectHashKeys [][]byte
+
+	recentMinuteDetectTimes  uint64
+	recentMinuteFailureTimes uint64
+
+	recentHourDetectTimes  uint64
+	recentHourFailureTimes uint64
+
+	recentDayDetectTimes  uint64
+	recentDayFailureTimes uint64
+}
+
+func (d *pegasusDetector) Start(rootCtx context.Context) error {
+	var err error
+	ctx, cancel := context.WithTimeout(rootCtx, 10*time.Second)
+	defer cancel()
+	d.detectTable, err = d.client.OpenTable(ctx, d.detectTableName)
+	if err != nil {
+		return err
+	}
+
+	ticker := time.NewTicker(d.detectInterval)
+	for {
+		select {
+		case <-rootCtx.Done(): // check if context cancelled
+			return nil
+		case <-ticker.C:
+			return nil
+		default:
+		}
+
+		// periodically set/get a configured Pegasus table.
+		d.detect(ctx)
+	}
+}
+
+func (d *pegasusDetector) detect(rootCtx context.Context) {
+	// TODO(yingchun): doesn't work, just to mute lint errors.
+	d.detectPartition(rootCtx, 1)
+}
+
+func (d *pegasusDetector) detectPartition(rootCtx context.Context, partitionIdx int) {
+	d.incrDetectTimes()
+
+	go func() {
+		ctx, cancel := context.WithTimeout(rootCtx, d.detectTimeout)
+		defer cancel()
+
+		hashkey := d.detectHashKeys[partitionIdx]
+		value := []byte("")
+
+		if err := d.detectTable.Set(ctx, hashkey, []byte(""), value); err != nil {
+			d.incrFailureTimes()
+			log.Errorf("set partition [%d] failed, hashkey=\"%s\": %s", partitionIdx, hashkey, err)
+		}
+		if _, err := d.detectTable.Get(ctx, hashkey, []byte("")); err != nil {
+			d.incrFailureTimes()
+			log.Errorf("get partition [%d] failed, hashkey=\"%s\": %s", partitionIdx, hashkey, err)
+		}
+	}()
+}
+
+func (d *pegasusDetector) incrDetectTimes() {
+	atomic.AddUint64(&d.recentMinuteDetectTimes, 1)
+	atomic.AddUint64(&d.recentHourDetectTimes, 1)
+	atomic.AddUint64(&d.recentDayDetectTimes, 1)
+}
+
+func (d *pegasusDetector) incrFailureTimes() {
+	atomic.AddUint64(&d.recentMinuteFailureTimes, 1)
+	atomic.AddUint64(&d.recentHourFailureTimes, 1)
+	atomic.AddUint64(&d.recentDayFailureTimes, 1)
+}
diff --git a/.github/workflows/module_labeler_conf.yml b/collector/config.yml
similarity index 58%
copy from .github/workflows/module_labeler_conf.yml
copy to collector/config.yml
index d5648edf9..b89993d6c 100644
--- a/.github/workflows/module_labeler_conf.yml
+++ b/collector/config.yml
@@ -14,39 +14,32 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
----
-github:
-  - .github/**/*
-admin-cli:
-  - admin-cli/**/*
-docker:
-  - docker/**/*
-go-client:
-  - go-client/**/*
-java-client:
-  - java-client/**/*
-nodejs-client:
-  - nodejs-client/**/*
-pegic:
-  - pegic/**/*
-python-client:
-  - python-client/**/*
-scala-client:
-  - scala-client/**/*
-thirdparty:
-  - thirdparty/**/*
-thrift:
-  - '**/*.thrift'
-docs:
-  - '**/*.md'
-scripts:
-  - 'scripts/**/*'
-  - '**/*.sh'
-build:
-  - 'cmake_modules/**/*'
-  - '**/CMakeLists.txt'
-  - 'run.sh'
-cpp:
-  # TODO(yingchun): add more fine-grained labels
-  - 'src/**/*.h'
-  - 'src/**/*.cpp'
+
+# the cluster that this collector is binding
+cluster_name : "onebox"
+
+# the meta server addresses of the cluster.
+meta_servers: 
+  - 127.0.0.1:34601
+  - 127.0.0.1:34602
+
+# local server port
+port : 34101
+
+metrics:
+  # use falcon as monitoring system.
+  sink : falcon 
+  report_interval : 10s
+
+prometheus:
+  # the exposed port for prometheus exposer
+  exposer_port : 1111 
+
+falcon_agent:
+  # the host IP of falcon agent
+  host : "127.0.0.1"
+  port : 1988
+  http_path : "/v1/push"
+
+available_detect:
+  table_name : test
diff --git a/collector/go.mod b/collector/go.mod
new file mode 100644
index 000000000..7288f318d
--- /dev/null
+++ b/collector/go.mod
@@ -0,0 +1,47 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+module github.com/pegasus-kv/collector
+
+go 1.13
+
+require (
+	github.com/ajg/form v1.5.1 // indirect
+	github.com/apache/incubator-pegasus/go-client v0.0.0-20220526071020-be5634371701
+	github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072 // indirect
+	github.com/google/go-querystring v1.0.0 // indirect
+	github.com/imkira/go-interpol v1.1.0 // indirect
+	github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 // indirect
+	github.com/kataras/iris/v12 v12.1.8
+	github.com/mattn/go-isatty v0.0.12 // indirect
+	github.com/moul/http2curl v1.0.0 // indirect
+	github.com/prometheus/client_golang v1.8.0
+	github.com/sergi/go-diff v1.1.0 // indirect
+	github.com/sirupsen/logrus v1.7.0
+	github.com/spf13/viper v1.7.1
+	github.com/stretchr/testify v1.6.1
+	github.com/tidwall/gjson v1.14.0
+	github.com/valyala/fasthttp v1.16.0 // indirect
+	github.com/xeipuuv/gojsonschema v1.2.0 // indirect
+	github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0 // indirect
+	github.com/yudai/gojsondiff v1.0.0 // indirect
+	github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82 // indirect
+	github.com/yudai/pp v2.0.1+incompatible // indirect
+	gopkg.in/natefinch/lumberjack.v2 v2.0.0
+	gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637
+	k8s.io/apimachinery v0.0.0-20191123233150-4c4803ed55e3
+)
diff --git a/collector/go.sum b/collector/go.sum
new file mode 100644
index 000000000..6efe8418b
--- /dev/null
+++ b/collector/go.sum
@@ -0,0 +1,739 @@
+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.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/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o=
+cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE=
+cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk=
+cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I=
+cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw=
+dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
+github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
+github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=
+github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
+github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
+github.com/CloudyKit/fastprinter v0.0.0-20200109182630-33d98a066a53 h1:sR+/8Yb4slttB4vD+b9btVEnWgL3Q00OBTzVT8B9C0c=
+github.com/CloudyKit/fastprinter v0.0.0-20200109182630-33d98a066a53/go.mod h1:+3IMCy2vIlbG1XG/0ggNQv0SvxCAIpPM5b1nCz56Xno=
+github.com/CloudyKit/jet/v3 v3.0.0 h1:1PwO5w5VCtlUUl+KTOBsTGZlhjWkcybsGaAau52tOy8=
+github.com/CloudyKit/jet/v3 v3.0.0/go.mod h1:HKQPgSJmdK8hdoAbKUUWajkHyHo4RaU5rMdUywE7VMo=
+github.com/Joker/hpp v1.0.0 h1:65+iuJYdRXv/XyN62C1uEmmOx3432rNG/rKlX6V7Kkc=
+github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY=
+github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0=
+github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ=
+github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
+github.com/PuerkitoBio/purell v1.0.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
+github.com/PuerkitoBio/urlesc v0.0.0-20160726150825-5bd2802263f2/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
+github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398 h1:WDC6ySpJzbxGWFh4aMxFFC28wwGp5pEuoTtvA4q/qQ4=
+github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0=
+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/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g=
+github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c=
+github.com/agiledragon/gomonkey v2.0.2+incompatible h1:eXKi9/piiC3cjJD1658mEE2o3NjkJ5vDLgYjCQu0Xlw=
+github.com/agiledragon/gomonkey v2.0.2+incompatible/go.mod h1:2NGfXu1a80LLr2cmWXGBDaHEjb1idR6+FVlX5T3D9hw=
+github.com/ajg/form v1.5.1 h1:t9c7v8JUKu/XxOGBU0yjNpaMloxGEJhUkqFRq0ibGeU=
+github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY=
+github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
+github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
+github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho=
+github.com/andybalholm/brotli v1.0.0 h1:7UCwP93aiSfvWpapti8g88vVVGp2qqtGyePsSuDafo4=
+github.com/andybalholm/brotli v1.0.0/go.mod h1:loMXtMfwqflxFJPmdbJO0a3KNoPuLBgiu3qAvBg8x/Y=
+github.com/apache/incubator-pegasus/go-client v0.0.0-20220526071020-be5634371701 h1:J9d8jaC0rKlnf8iQuImyBiDrZnl794JlGMXX8DhMNrs=
+github.com/apache/incubator-pegasus/go-client v0.0.0-20220526071020-be5634371701/go.mod h1:vOnSzVwVYmatouq8qEBX+yWV4AvoCTLRpQn6JA+qojs=
+github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
+github.com/apache/thrift v0.13.0 h1:5hryIiq9gtn+MiLVn0wP37kb/uTeRZgN08WoCsAhIhI=
+github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
+github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o=
+github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8=
+github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY=
+github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
+github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A=
+github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU=
+github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
+github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g=
+github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible h1:Ppm0npCCsmuR9oQaBtRuZcmILVE74aXE+AmrJj8L2ns=
+github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g=
+github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
+github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
+github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
+github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
+github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
+github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84=
+github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ=
+github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4=
+github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM=
+github.com/cenkalti/backoff/v4 v4.1.0 h1:c8LkOFQTzuO0WBM/ae5HdGQuZPfPxp7lqBRwQRm4fSc=
+github.com/cenkalti/backoff/v4 v4.1.0/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw=
+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/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY=
+github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
+github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE=
+github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
+github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
+github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI=
+github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk=
+github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
+github.com/coreos/etcd v3.3.13+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-semver v0.3.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/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
+github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
+github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
+github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE=
+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/davecgh/go-spew v0.0.0-20151105211317-5215b55f46b2/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+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/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4=
+github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
+github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw=
+github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no=
+github.com/docker/spdystream v0.0.0-20160310174837-449fdfce4d96/go.mod h1:Qh8CwZgvJUkLughtfhJv5dyTYa91l1fOUCrgjqmcifM=
+github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
+github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
+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/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M=
+github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o=
+github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM=
+github.com/elazarl/goproxy v0.0.0-20170405201442-c4fc26588b6e/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc=
+github.com/emicklei/go-restful v0.0.0-20170410110728-ff4f55a20633/go.mod h1:otzb+WCGbkyDHkqmQmT5YD2WR4BBwUdeQoFo8l/7tVs=
+github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g=
+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/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw=
+github.com/evanphx/json-patch v4.2.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk=
+github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072 h1:DddqAaWDpywytcG8w/qoQ5sAN8X12d3Z3koB0C3Rxsc=
+github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8=
+github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
+github.com/fatih/structs v1.1.0 h1:Q7juDM0QtcnhCpeyLGQKyg4TOIghuNXrkL32pHAUMxo=
+github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M=
+github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw=
+github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g=
+github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4=
+github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20=
+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/gavv/httpexpect v2.0.0+incompatible h1:1X9kcRshkSKEjNJJxX9Y9mQ5BRfbxU5kORdjhlA1yX8=
+github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc=
+github.com/ghodss/yaml v0.0.0-20150909031657-73d445a93680/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
+github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
+github.com/go-check/check v0.0.0-20180628173108-788fd7840127 h1:0gkP6mzaMqkmpcJYCFOLkIBwI7xFExG03bbkOkCvUPI=
+github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98=
+github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
+github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o=
+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-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
+github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas=
+github.com/go-openapi/jsonpointer v0.0.0-20160704185906-46af16f9f7b1/go.mod h1:+35s3my2LFTysnkMfxsJBAMHj/DoqoB9knIWoYG/Vk0=
+github.com/go-openapi/jsonreference v0.0.0-20160704190145-13c6e3589ad9/go.mod h1:W3Z9FmVs9qj+KR4zFKmDPGiLdk1D9Rlm7cyMvf57TTg=
+github.com/go-openapi/spec v0.0.0-20160808142527-6aced65f8501/go.mod h1:J8+jY1nAiCcj+friV/PDoE1/3eeccG9LYBs0tYvLOWc=
+github.com/go-openapi/swag v0.0.0-20160704191624-1d0bd113de87/go.mod h1:DXUve3Dpr1UfpPtxFw+EFuQ41HhCWZfha5jSVRG7C7I=
+github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w=
+github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
+github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo=
+github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw=
+github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM=
+github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s=
+github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
+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/gogo/protobuf v1.2.2-0.20190723190241-65acae22fc9d/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
+github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
+github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
+github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
+github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/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 v0.0.0-20161109072736-4bd1920723d7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
+github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
+github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
+github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
+github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
+github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM=
+github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
+github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4=
+github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
+github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
+github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
+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-querystring v1.0.0 h1:Xkwi/a1rcvNg1PPYe5vI8GbeBY/jrVuDX5ASuANWTrk=
+github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
+github.com/google/gofuzz v0.0.0-20161122191042-44d81051d367/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI=
+github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
+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/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
+github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/google/uuid v1.1.1/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/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk=
+github.com/googleapis/gnostic v0.0.0-20170729233727-0c5108395e2d/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY=
+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/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg=
+github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs=
+github.com/gorilla/mux v1.7.3/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.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
+github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc=
+github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
+github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
+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.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
+github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
+github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q=
+github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE=
+github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8=
+github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8=
+github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
+github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80=
+github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
+github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
+github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk=
+github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU=
+github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU=
+github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4=
+github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
+github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
+github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA=
+github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90=
+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/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64=
+github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ=
+github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
+github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc=
+github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
+github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
+github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg=
+github.com/imkira/go-interpol v1.1.0 h1:KIiKr0VSG2CUW1hl1jpiyuzuJeKUUpC8iM1AIE7N1Vk=
+github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA=
+github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
+github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo=
+github.com/iris-contrib/blackfriday v2.0.0+incompatible h1:o5sHQHHm0ToHUlAJSTjW9UWicjJSDDauOOQ2AHuIVp4=
+github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI=
+github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0=
+github.com/iris-contrib/jade v1.1.3 h1:p7J/50I0cjo0wq/VWVCDFd8taPJbuFC+bq23SniRFX0=
+github.com/iris-contrib/jade v1.1.3/go.mod h1:H/geBymxJhShH5kecoiOCSssPX7QWYH7UaeZTSWddIk=
+github.com/iris-contrib/pongo2 v0.0.1 h1:zGP7pW51oi5eQZMIlGA3I+FHY9/HOQWDB+572yin0to=
+github.com/iris-contrib/pongo2 v0.0.1/go.mod h1:Ssh+00+3GAZqSQb30AvBRNxBx7rf0GqwkjqxNd0u65g=
+github.com/iris-contrib/schema v0.0.1 h1:10g/WnoRR+U+XXHWKBHeNy/+tZmM2kcAVGLOsz+yaDA=
+github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw=
+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/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
+github.com/json-iterator/go v0.0.0-20180612202835-f2b4162afba3/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
+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/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.10 h1:Kz6Cvnvv2wGdaG/V8yMvfkmNiXq9Ya2KUv4rouJJr68=
+github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
+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/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM=
+github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 h1:uC1QfSlInpQF+M0ao65imhwqKnz3Q2z/d8PWZRMQvDM=
+github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k=
+github.com/kataras/golog v0.0.10 h1:vRDRUmwacco/pmBAm8geLn8rHEdc+9Z4NAr5Sh7TG/4=
+github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8=
+github.com/kataras/iris/v12 v12.1.8 h1:O3gJasjm7ZxpxwTH8tApZsvf274scSGQAUpNe47c37U=
+github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE=
+github.com/kataras/neffos v0.0.14/go.mod h1:8lqADm8PnbeFfL7CLXh1WHw53dG27MC3pgi2R1rmoTE=
+github.com/kataras/pio v0.0.2 h1:6NAi+uPJ/Zuid6mrAKlgpbI11/zK/lV4B2rxWaJN98Y=
+github.com/kataras/pio v0.0.2/go.mod h1:hAoW0t9UmXi4R5Oyq5Z4irTbaTsOemSrDGUtaTl7Dro=
+github.com/kataras/sitemap v0.0.5 h1:4HCONX5RLgVy6G4RkYOV3vKNcma9p236LdGOipJsaFE=
+github.com/kataras/sitemap v0.0.5/go.mod h1:KY2eugMKiPwsJgx7+U103YZehfvNGOXURubcGyk0Bz8=
+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.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
+github.com/klauspost/compress v1.10.7 h1:7rix8v8GpI3ZBb0nSozFRgbtXKv+hOe+qfEpZqybrAg=
+github.com/klauspost/compress v1.10.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs=
+github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/konsorten/go-windows-terminal-sequences v1.0.3/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/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/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM=
+github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4=
+github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ=
+github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ=
+github.com/magiconair/properties v1.8.1 h1:ZC2Vc7/ZFkGmsVC9KvOjumD+G5lXy2RtTKyzRKO2BQ4=
+github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ=
+github.com/mailru/easyjson v0.0.0-20160728113105-d5b7844b561a/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
+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-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
+github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
+github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY=
+github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
+github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
+github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw=
+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/mediocregopher/radix/v3 v3.4.2/go.mod h1:8FL3F6UQRXHXIBSPUs5h0RybMF8i4n7wVopoX3x7Bv8=
+github.com/microcosm-cc/bluemonday v1.0.2 h1:5lPfLTTAvAbtS0VqT+94yOtFnGfUWYyx0+iToC3Os3s=
+github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc=
+github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
+github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc=
+github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
+github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
+github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI=
+github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg=
+github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY=
+github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/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/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
+github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg=
+github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
+github.com/modern-go/reflect2 v0.0.0-20180320133207-05fbef0ca5da/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
+github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
+github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI=
+github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
+github.com/moul/http2curl v1.0.0 h1:dRMWoAtb+ePxMlLkrCbAqh4TlPHXvoGUSQ323/9Zahs=
+github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ=
+github.com/munnerz/goautoneg v0.0.0-20120707110453-a547fc61f48d/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ=
+github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
+github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
+github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f/go.mod h1:ZdcZmHo+o7JKHSa8/e818NopupXU1YMK5fe1lsApnBw=
+github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg=
+github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU=
+github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k=
+github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w=
+github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
+github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
+github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c=
+github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs=
+github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA=
+github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U=
+github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo=
+github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
+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/ginkgo v1.10.1 h1:q/mM8GF/n0shIN8SaAZ0V+jnLPzen6WIVZdiwrRlMlo=
+github.com/onsi/ginkgo v1.10.1/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
+github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
+github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
+github.com/onsi/gomega v1.7.0 h1:XPnZz8VVBHjVsy1vzJmRwIcSwiUO+JFfrv/xGiigmME=
+github.com/onsi/gomega v1.7.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
+github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk=
+github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis=
+github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74=
+github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
+github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
+github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5/go.mod h1:/wsWhb9smxSfWAKL3wpBW7V8scJMt8N8gnaMCS9E/cA=
+github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw=
+github.com/openzipkin/zipkin-go v0.2.1/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4=
+github.com/openzipkin/zipkin-go v0.2.2/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4=
+github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIwwtUjcrb0b5/5kLM=
+github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
+github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k=
+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/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac=
+github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc=
+github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
+github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA=
+github.com/pmezard/go-difflib v0.0.0-20151028094244-d8ed2627bdf0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+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/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI=
+github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
+github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs=
+github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso=
+github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
+github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og=
+github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M=
+github.com/prometheus/client_golang v1.8.0 h1:zvJNkoCFAnYFNC24FV8nW4JdRJ3GIFcLbg65lL/JDcw=
+github.com/prometheus/client_golang v1.8.0/go.mod h1:O9VU6huf47PktckDQfMTX0Y8tY0/7TSWwj+ITvv0TnM=
+github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
+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/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M=
+github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
+github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA=
+github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo=
+github.com/prometheus/common v0.14.0 h1:RHRyE8UocrbjU+6UvRzwi6HjiDfxrrBU91TtbKzkGp4=
+github.com/prometheus/common v0.14.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s=
+github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
+github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
+github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
+github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
+github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A=
+github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
+github.com/prometheus/procfs v0.2.0 h1:wH4vA7pcjKuZzjF7lM8awk4fnuJO6idemZXoKnULUx4=
+github.com/prometheus/procfs v0.2.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
+github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU=
+github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
+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/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g=
+github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
+github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
+github.com/ryanuber/columnize v2.1.0+incompatible h1:j1Wcmh8OrK4Q7GXY+V7SVSY8nUWQxHW5TkBe7YUl+2s=
+github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
+github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E=
+github.com/schollz/closestmatch v2.1.0+incompatible h1:Uel2GXEpJqOWBrlyI+oY9LTiyyjYS17cCYRqP13/SHk=
+github.com/schollz/closestmatch v2.1.0+incompatible/go.mod h1:RtP1ddjLong6gTkbtmuhtR2uUrrJOpYzYRvbcPAid+g=
+github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc=
+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/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/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
+github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
+github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88=
+github.com/sirupsen/logrus v1.7.0 h1:ShrD1U9pZB12TX0cVy0DtePoCH97K8EtX+mg7ZARUtM=
+github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0=
+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 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/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY=
+github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/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.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/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ=
+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 v0.0.0-20170130214245-9ff6c6923cff/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA=
+github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg=
+github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s=
+github.com/spf13/viper v1.7.1 h1:pM5oEahlgWv/WnHXpgbKz7iLIxRf65tye2Ci+XFK5sk=
+github.com/spf13/viper v1.7.1/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg=
+github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw=
+github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw=
+github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI=
+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 v0.0.0-20151208002404-e3a8ff8ce365/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
+github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
+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/subosito/gotenv v1.2.0 h1:Slr1R9HxAlEKefgq5jn9U+DnETlIUa6HfgEzj0g5d7s=
+github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw=
+github.com/tidwall/gjson v1.14.0 h1:6aeJ0bzojgWLa82gDQHcx3S0Lr/O51I9bJ5nv6JFx5w=
+github.com/tidwall/gjson v1.14.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk=
+github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA=
+github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM=
+github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs=
+github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
+github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0=
+github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
+github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
+github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6KllzawFIhcdPw=
+github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
+github.com/valyala/fasthttp v1.16.0 h1:9zAqOYLl8Tuy3E5R6ckzGDJ1g8+pw15oQp2iL9Jl6gQ=
+github.com/valyala/fasthttp v1.16.0/go.mod h1:YOKImeEosDdBPnxc0gy7INqi3m1zK6A+xl6TwOBhHCA=
+github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
+github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f h1:J9EGpcZtP0E/raorCMxlFGSTBrsSlaDGf3jU/qvAE2c=
+github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU=
+github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415 h1:EzJWgHovont7NscjpAxXsDA8S8BMYve8Y5+7cuRE7R0=
+github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ=
+github.com/xeipuuv/gojsonschema v1.2.0 h1:LhYJRs+L4fBtjZUfuSZIKGeVu0QRy8e5Xi7D17UxZ74=
+github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y=
+github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
+github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q=
+github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0 h1:6fRhSjgLCkTD3JnJxvaJ4Sj+TYblw757bqYgZaOq5ZY=
+github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI=
+github.com/yudai/gojsondiff v1.0.0 h1:27cbfqXLVEJ1o8I6v3y9lg8Ydm53EKqHXAOMxEGlCOA=
+github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg=
+github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82 h1:BHyfKlQyqbsFN5p3IfnEUduWvb9is428/nNb5L3U01M=
+github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM=
+github.com/yudai/pp v2.0.1+incompatible h1:Q4//iY4pNF6yPLZIigmvcl7k/bPgrcTPIFIcmawg5bI=
+github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc=
+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-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg=
+go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
+go.opencensus.io v0.20.2/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/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
+go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
+go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
+go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
+go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
+go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
+go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA=
+go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
+go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
+golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
+golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
+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/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20191227163750-53104e6ec876/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI=
+golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/exp v0.0.0-20190121172915-509febef88a4/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/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-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/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+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-20170114055629-f2499483f923/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+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-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-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-20190327091125-710a502c58a2/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/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-20191004110552-13f9640d40b9/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20191105084925-a882066a44e0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20200602114024-627f9648deb9/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
+golang.org/x/net v0.0.0-20200625001655-4c5254603344 h1:vGXIOMxbNfDTk/aXCmfdLgkrSV+Z2tcbze+pEc3v5W4=
+golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
+golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/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/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/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/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sys v0.0.0-20170830134202-bb24a47a89ea/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/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-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/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-20190215142949-d0b11bdaac8a/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-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-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20201015000850-e3ed0017c211 h1:9UQO31fZ+0aKQOFldThf7BKPMJTiBfWycGh/u3UoO88=
+golang.org/x/sys v0.0.0-20201015000850-e3ed0017c211/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
+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/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/time v0.0.0-20191024005414-555d28b269f0/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-20180828015842-6cd1fcedba52/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-20181011042414-1f849cf54d09/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-20181221001348-537d06c36207/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-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-20190327201419-c70d86f8b7cf/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-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/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/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=
+google.golang.org/api v0.3.1/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/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.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/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
+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-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s=
+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/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
+google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
+google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM=
+google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
+google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
+google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
+google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
+google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
+google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
+google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
+google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
+google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
+google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
+google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
+google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
+google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM=
+google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
+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/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+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 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4=
+gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
+gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o=
+gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
+gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
+gopkg.in/ini.v1 v1.51.1 h1:GyboHr4UqMiLUybYjd22ZjQIKEJEpgtLXtuGbR21Oho=
+gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
+gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA=
+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 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
+gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
+gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637 h1:yiW+nvdHb9LVqSHQBXfZCieqV4fzYhNBql77zY0ykqs=
+gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637/go.mod h1:BHsqpu/nsuzkT5BpiH1EMZPLyqSMM8JbIavyFACoFNk=
+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/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU=
+gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v3 v3.0.0-20191120175047-4206685974f2/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
+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-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/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg=
+k8s.io/apimachinery v0.0.0-20191123233150-4c4803ed55e3 h1:FErmbNIJruD5GT2oVEjtPn5Ar5+rcWJsC8/PPUkR0s4=
+k8s.io/apimachinery v0.0.0-20191123233150-4c4803ed55e3/go.mod h1:b9qmWdKlLuU9EBh+06BtLcSf/Mu89rWL33naRxs1uZg=
+k8s.io/gengo v0.0.0-20190128074634-0689ccc1d7d6/go.mod h1:ezvh/TsK7cY6rbqRK0oQQ8IAqLxYwwyPxAX1Pzy0ii0=
+k8s.io/klog v0.0.0-20181102134211-b9b56d5dfc92/go.mod h1:Gq+BEi5rUBO/HRz0bTSXDUcqjScdoY3a9IHpCEIOOfk=
+k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I=
+k8s.io/kube-openapi v0.0.0-20191107075043-30be4d16710a/go.mod h1:1TqjTSzOxsLGIKfj0lK8EeCP7K1iUG65v09OM0/WG5E=
+rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8=
+sigs.k8s.io/structured-merge-diff v0.0.0-20190525122527-15d366b2352e/go.mod h1:wWxsB5ozmmv/SG7nM11ayaAW51xMvak/t1r0CSlcokI=
+sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o=
+sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU=
diff --git a/collector/hotspot/algo.go b/collector/hotspot/algo.go
new file mode 100644
index 000000000..6b24419cf
--- /dev/null
+++ b/collector/hotspot/algo.go
@@ -0,0 +1,18 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package hotspot
diff --git a/collector/main.go b/collector/main.go
new file mode 100644
index 000000000..d3d45d26c
--- /dev/null
+++ b/collector/main.go
@@ -0,0 +1,99 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package main
+
+import (
+	"errors"
+	"fmt"
+	"os"
+	"os/signal"
+	"runtime"
+	"strings"
+	"syscall"
+
+	"github.com/pegasus-kv/collector/aggregate"
+	"github.com/pegasus-kv/collector/usage"
+	"github.com/pegasus-kv/collector/webui"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+	"gopkg.in/natefinch/lumberjack.v2"
+	"gopkg.in/tomb.v2"
+)
+
+// callerPrettifier simplifies the caller info
+func callerPrettifier(f *runtime.Frame) (function string, file string) {
+	function = f.Function[strings.LastIndex(f.Function, "/")+1:]
+	file = fmt.Sprint(f.File[strings.LastIndex(f.File, "/")+1:], ":", f.Line)
+	return function, file
+}
+
+// setupSignalHandler setup signal handler for collector
+func setupSignalHandler(shutdownFunc func()) {
+	closeSignalChan := make(chan os.Signal, 1)
+	signal.Notify(closeSignalChan,
+		syscall.SIGHUP,
+		syscall.SIGINT,
+		syscall.SIGTERM,
+		syscall.SIGQUIT)
+
+	go func() {
+		sig := <-closeSignalChan
+		log.Infof("got signal %s to exit", sig.String())
+		shutdownFunc()
+	}()
+}
+
+func main() {
+	// initialize logging
+	log.SetFormatter(&log.TextFormatter{
+		DisableColors:    true,
+		FullTimestamp:    true,
+		CallerPrettyfier: callerPrettifier,
+	})
+	log.SetOutput(&lumberjack.Logger{ // rolling log
+		Filename:  "./pegasus.log",
+		MaxSize:   50, // MegaBytes
+		MaxAge:    2,  // days
+		LocalTime: true,
+	})
+	log.SetReportCaller(true)
+
+	// TODO(wutao1): use args[1] as config path
+	viper.SetConfigFile("config.yml")
+	viper.SetConfigType("yaml")
+	if err := viper.ReadInConfig(); err != nil {
+		log.Fatal("failed to read config: ", err)
+		return
+	}
+
+	webui.StartWebServer()
+
+	tom := &tomb.Tomb{}
+	setupSignalHandler(func() {
+		tom.Kill(errors.New("collector terminates")) // kill other goroutines
+	})
+	tom.Go(func() error {
+		aggregate.Start(tom)
+		return nil
+	})
+	tom.Go(func() error {
+		usage.NewTableUsageRecorder().Start(tom)
+		return nil
+	})
+	<-tom.Dead() // gracefully wait until all goroutines dead
+}
diff --git a/collector/metrics/falcon_sink.go b/collector/metrics/falcon_sink.go
new file mode 100644
index 000000000..4ccaedfcb
--- /dev/null
+++ b/collector/metrics/falcon_sink.go
@@ -0,0 +1,144 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package metrics
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"net/http"
+	"time"
+
+	"github.com/pegasus-kv/collector/aggregate"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+)
+
+type falconConfig struct {
+	falconAgentHost     string
+	falconAgentPort     uint32
+	falconAgentHTTPPath string
+
+	clusterName           string
+	port                  uint32
+	metricsReportInterval time.Duration
+}
+
+type falconSink struct {
+	cfg *falconConfig
+}
+
+type falconMetricData struct {
+	Endpoint    string  // the cluster name
+	Metric      string  // metric name
+	Timestamp   int64   // the reporting time in unix seconds
+	Step        int32   // the reporting time interval in seconds
+	Value       float64 // metric value
+	CounterType string  // GAUGE or COUNTER
+	Tags        string
+}
+
+func newFalconSink() *falconSink {
+	sink := &falconSink{}
+	sink.cfg = &falconConfig{
+		falconAgentHost:       viper.GetString("falcon_agent.host"),
+		falconAgentPort:       viper.GetUint32("falcon_agent.port"),
+		falconAgentHTTPPath:   viper.GetString("falcon_agent.http_path"),
+		clusterName:           viper.GetString("cluster_name"),
+		port:                  viper.GetUint32("port"),
+		metricsReportInterval: viper.GetDuration("metrics.report_interval"),
+	}
+	return sink
+}
+
+func (m *falconMetricData) setData(name string, value float64, tags map[string]string) {
+	m.Metric = name
+	m.Value = value
+	m.Timestamp = time.Now().Unix()
+
+	firstTag := true
+	for k, v := range tags {
+		if firstTag {
+			firstTag = false
+		} else {
+			m.Tags += ","
+		}
+		m.Tags += k + "=" + v
+	}
+}
+
+func (m *falconMetricData) toJSON() []byte {
+	result, err := json.Marshal(m)
+	if err != nil {
+		log.Fatal("failed to marshall falcon metric to json: ", err)
+	}
+	return result
+}
+
+type falconDataSerializer struct {
+	buf *bytes.Buffer
+
+	mdata *falconMetricData
+}
+
+func (s *falconDataSerializer) serialize(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+	s.buf.WriteString("[")
+	for _, table := range stats {
+		for name, value := range table.Stats {
+			s.mdata.setData(name, value, map[string]string{
+				"entity": "table",
+				"table":  name,
+			})
+		}
+		s.buf.Write(s.mdata.toJSON())
+	}
+	for name, value := range allStats.Stats {
+		s.mdata.setData(name, value, map[string]string{
+			"entity": "cluster",
+		})
+		s.buf.Write(s.mdata.toJSON())
+	}
+}
+
+func (sink *falconSink) Report(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+	serializer := &falconDataSerializer{
+		buf: bytes.NewBuffer(nil),
+	}
+	serializer.mdata = &falconMetricData{
+		Endpoint:    sink.cfg.clusterName,
+		Step:        int32(sink.cfg.metricsReportInterval.Seconds()),
+		CounterType: "GAUGE",
+	}
+	serializer.serialize(stats, allStats)
+	sink.postHTTPData(serializer.buf.Bytes())
+}
+
+func (sink *falconSink) postHTTPData(data []byte) {
+	url := fmt.Sprintf("http://%s:%d/%s", sink.cfg.falconAgentHost, sink.cfg.falconAgentPort, sink.cfg.falconAgentHTTPPath)
+	resp, err := http.Post(url, "application/x-www-form-urlencoded", bytes.NewReader([]byte(data)))
+	if err == nil && resp.StatusCode != http.StatusOK {
+		err = errors.New(resp.Status)
+	}
+	if err != nil {
+		log.Errorf("failed to post metrics to falcon agent: %s", err)
+		return
+	}
+
+	defer resp.Body.Close()
+}
diff --git a/collector/metrics/prometheus_sink.go b/collector/metrics/prometheus_sink.go
new file mode 100644
index 000000000..e37fbc33d
--- /dev/null
+++ b/collector/metrics/prometheus_sink.go
@@ -0,0 +1,110 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package metrics
+
+import (
+	"sync"
+
+	"github.com/pegasus-kv/collector/aggregate"
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+type prometheusMetricFamily struct {
+	metrics map[string]prometheus.Gauge
+}
+
+func (f *prometheusMetricFamily) set(name string, value float64) {
+	f.metrics[name].Set(value)
+}
+
+type prometheusSink struct {
+	tableMap map[int]*prometheusMetricFamily
+
+	tableLock sync.RWMutex
+
+	clusterMetric *prometheusMetricFamily
+
+	allTrackedMetrics []string
+}
+
+func newPrometheusSink() *prometheusSink {
+	sink := &prometheusSink{
+		tableMap:          make(map[int]*prometheusMetricFamily),
+		allTrackedMetrics: aggregate.AllMetrics(),
+	}
+	sink.clusterMetric = sink.newClusterMetricFamily()
+
+	aggregate.AddHookAfterTableDropped(func(appID int) {
+		// remove the metrics family belongs to the table
+		sink.tableLock.Lock()
+		for _, gauge := range sink.tableMap[appID].metrics {
+			prometheus.Unregister(gauge)
+		}
+		delete(sink.tableMap, appID)
+		sink.tableLock.Unlock()
+	})
+	return sink
+}
+
+func (sink *prometheusSink) Report(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+	for _, table := range stats {
+		sink.tableLock.Lock()
+		defer sink.tableLock.Unlock()
+
+		var mfamily *prometheusMetricFamily
+		var found bool
+		if mfamily, found = sink.tableMap[table.AppID]; !found {
+			mfamily = sink.newTableMetricFamily(table.TableName)
+			// insert table metrics family
+			sink.tableMap[table.AppID] = mfamily
+		}
+		fillStatsIntoGauges(table.Stats, mfamily)
+	}
+	fillStatsIntoGauges(allStats.Stats, sink.clusterMetric)
+}
+
+func fillStatsIntoGauges(stats map[string]float64, family *prometheusMetricFamily) {
+	for name, value := range stats {
+		family.set(name, value)
+	}
+}
+
+func (sink *prometheusSink) newTableMetricFamily(tableName string) *prometheusMetricFamily {
+	return sink.newMetricFamily(map[string]string{"table": tableName, "entity": "table"})
+}
+
+func (sink *prometheusSink) newClusterMetricFamily() *prometheusMetricFamily {
+	return sink.newMetricFamily(map[string]string{"entity": "cluster"})
+}
+
+func (sink *prometheusSink) newMetricFamily(labels map[string]string) *prometheusMetricFamily {
+	mfamily := &prometheusMetricFamily{
+		metrics: make(map[string]prometheus.Gauge),
+	}
+	for _, m := range sink.allTrackedMetrics {
+		// create and register a gauge
+		opts := prometheus.GaugeOpts{
+			Name:        m,
+			ConstLabels: labels,
+		}
+		gauge := prometheus.NewGauge(opts)
+		prometheus.MustRegister(gauge)
+		mfamily.metrics[m] = gauge
+	}
+	return mfamily
+}
diff --git a/collector/metrics/sink.go b/collector/metrics/sink.go
new file mode 100644
index 000000000..fbe590e86
--- /dev/null
+++ b/collector/metrics/sink.go
@@ -0,0 +1,53 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package metrics
+
+import (
+	"github.com/pegasus-kv/collector/aggregate"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+)
+
+// Sink is the destination where the metrics are reported to.
+type Sink interface {
+
+	// Report the snapshot of metrics to the monitoring system. The report can possibly fail.
+	Report(stats []aggregate.TableStats, allStats aggregate.ClusterStats)
+}
+
+// NewSink creates a Sink which reports metrics to the configured monitoring system.
+func NewSink() Sink {
+	var sink Sink
+	cfgSink := viper.Get("metrics.sink")
+	if cfgSink == "falcon" {
+		sink = newFalconSink()
+	} else if cfgSink == "prometheus" {
+		sink = newPrometheusSink()
+	} else {
+		log.Fatalf("invalid metrics_sink = %s", cfgSink)
+		return nil
+	}
+
+	aggregate.AddHookAfterTableStatEmitted(func(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+		go func() {
+			sink.Report(stats, allStats)
+		}()
+	})
+
+	return sink
+}
diff --git a/collector/templates/index.html b/collector/templates/index.html
new file mode 100644
index 000000000..fc3de7aa1
--- /dev/null
+++ b/collector/templates/index.html
@@ -0,0 +1,80 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+-->
+
+<!DOCTYPE html>
+<html>
+
+<head>
+    <meta charset="utf-8">
+    <!--responsive support-->
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <title>Pegasus Collector</title>
+    <link rel="stylesheet" href="https://cdn.jsdelivr.net/npm/bulma@0.9.0/css/bulma.min.css">
+</head>
+
+<nav class="navbar is-light" style="padding-left: 2rem;">
+    <div class="navbar-brand">
+        <a class="navbar-item" href="/"><strong>Pegasus Collector</strong></a>
+    </div>
+</nav>
+<main>
+    <div class="container" style="padding: 2rem;">
+        <h1 class="title" style="padding-bottom: 1rem;">Cluster Stats</h1>
+        <table class="table is-bordered is-striped is-fullwidth">
+            <thead>
+                <tr>
+                    <th>Perf-Counter</th>
+                    {{range .PerfIDs}}
+                    <th>{{.}}</th>
+                    {{end}}
+                </tr>
+            </thead>
+            <tbody>
+                {{range .PerfCounters}}
+                <tr>
+                    <td>{{.PerfCounter}}</td>
+                    {{range .Values}}
+                    <td>{{.}}</td>
+                    {{end}}
+                </tr>
+                {{end}}
+            </tbody>
+        </table>
+    </div>
+    <div class="container" style="padding: 2rem;">
+        <h1 class="title" style="padding-bottom: 1rem;">Tables</h1>
+        <table class="table is-bordered is-striped is-fullwidth">
+            <thead>
+                <tr>
+                    <th>Table Name</th>
+                    <th>Link</th>
+                </tr>
+            </thead>
+            <tbody>
+                {{range .Tables}}
+                <tr>
+                    <td>{{.TableName}}</td>
+                </tr>
+                {{end}}
+            </tbody>
+        </table>
+    </div>
+</main>
+
+</html>
\ No newline at end of file
diff --git a/collector/usage/usage_recorder.go b/collector/usage/usage_recorder.go
new file mode 100644
index 000000000..6a6784d39
--- /dev/null
+++ b/collector/usage/usage_recorder.go
@@ -0,0 +1,115 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package usage
+
+import (
+	"context"
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/pegasus"
+	"github.com/pegasus-kv/collector/aggregate"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+	"gopkg.in/tomb.v2"
+)
+
+// TableUsageRecorder records the usage of each table into a Pegasus table.
+// The usage statistics can be used for service cost calculation.
+type TableUsageRecorder interface {
+
+	// Start recording until the ctx cancelled. This method will block the current thread.
+	Start(tom *tomb.Tomb)
+}
+
+// NewTableUsageRecorder returns an instance of TableUsageRecorder
+func NewTableUsageRecorder() TableUsageRecorder {
+	return &tableUsageRecorder{
+		usageStatApp: viper.GetString("usage_stat_app"),
+	}
+}
+
+type tableUsageRecorder struct {
+	client pegasus.Client
+	table  pegasus.TableConnector
+
+	usageStatApp string
+}
+
+func (rec *tableUsageRecorder) Start(tom *tomb.Tomb) {
+	if rec.usageStatApp == "" {
+		// if no stat app is specified, usage recorder is considered as disabled.
+		return
+	}
+
+	metaServer := viper.GetString("meta_server")
+	rec.client = pegasus.NewClient(pegasus.Config{MetaServers: []string{metaServer}})
+	for {
+		var err error
+		rec.table, err = rec.client.OpenTable(tom.Context(context.TODO()), rec.usageStatApp)
+		if err != nil {
+			// retry indefinitely
+			log.Errorf("failed to open table: %s", err.Error())
+			sleepWait(tom, 15*time.Second)
+			continue
+		}
+		break
+	}
+
+	aggregate.AddHookAfterTableStatEmitted(func(stats []aggregate.TableStats, allStat aggregate.ClusterStats) {
+		rootCtx := tom.Context(context.TODO())
+		for _, s := range stats {
+			rec.writeTableUsage(rootCtx, &s)
+		}
+	})
+}
+
+func sleepWait(tom *tomb.Tomb, waitTime time.Duration) {
+	ticker := time.NewTicker(waitTime)
+	select {
+	case <-tom.Dying():
+		return
+	case <-ticker.C:
+	}
+}
+
+func (rec *tableUsageRecorder) writeTableUsage(ctx context.Context, tb *aggregate.TableStats) {
+	hashKey := []byte(fmt.Sprintf("%d", tb.Timestamp.Unix()))
+	sortkey := []byte("cu")
+
+	readCU := tb.Stats["recent_read_cu"]
+	writeCU := tb.Stats["recent_write_cu"]
+	value := []byte(fmt.Sprintf("{\"%d\":[%f, %f]}", tb.AppID, readCU, writeCU))
+
+	go func() {
+		maxRetryCount := 10
+		for maxRetryCount > 0 {
+			// TODO(wutao): set rpc timeout
+			err := rec.table.Set(ctx, hashKey, sortkey, value)
+			if err == nil {
+				break
+			}
+			log.Errorf("failed to write cu [timestamp: %s, appid: %d, readcu: %f, writecu: %f]",
+				tb.Timestamp.Local().String(),
+				tb.AppID,
+				readCU,
+				writeCU)
+			maxRetryCount--
+		}
+	}()
+}
diff --git a/collector/webui/index.go b/collector/webui/index.go
new file mode 100644
index 000000000..0c0d26ff4
--- /dev/null
+++ b/collector/webui/index.go
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package webui
+
+import (
+	"github.com/kataras/iris/v12"
+	"github.com/pegasus-kv/collector/aggregate"
+)
+
+var indexPageClusterStats = []string{
+	"write_bytes",
+	"read_bytes",
+	"write_qps",
+	"read_qps",
+}
+
+func renderIndexClusterCharts(ctx iris.Context) {
+	type perfCounterHTML struct {
+		PerfCounter string
+		Values      []float64
+	}
+	var PerfCounters []*perfCounterHTML
+
+	snapshots := aggregate.SnapshotClusterStats()
+	for _, s := range indexPageClusterStats {
+		PerfCounters = append(PerfCounters, &perfCounterHTML{
+			PerfCounter: s,
+		})
+		p := PerfCounters[len(PerfCounters)-1]
+		for _, snapshot := range snapshots {
+			if v, found := snapshot.Stats[s]; found {
+				p.Values = append(p.Values, v)
+			}
+		}
+	}
+	ctx.ViewData("PerfCounters", PerfCounters)
+
+	var PerfIDs []string
+	for _, sn := range snapshots {
+		PerfIDs = append(PerfIDs, sn.Timestamp.Format("15:04:00"))
+	}
+	ctx.ViewData("PerfIDs", PerfIDs)
+}
+
+func indexHandler(ctx iris.Context) {
+	renderIndexClusterCharts(ctx)
+
+	// metaClient := client(viper.GetString("meta_server"))
+	// tables, err := metaClient.ListTables()
+	// if err != nil {
+	// 	ctx.ResponseWriter().WriteString("Failed to list tables from MetaServer")
+	// 	ctx.StatusCode(iris.StatusInternalServerError)
+	// 	return
+	// }
+	// type tableHTMLRow struct {
+	// 	TableName string
+	// 	Link      string
+	// }
+	// var Tables []tableHTMLRow
+	// for _, tb := range tables {
+	// 	Tables = append(Tables, tableHTMLRow{TableName: tb.TableName})
+	// }
+	// ctx.ViewData("Tables", Tables)
+
+	err := ctx.View("index.html")
+	if err != nil {
+		return
+	}
+}
diff --git a/collector/webui/tables.go b/collector/webui/tables.go
new file mode 100644
index 000000000..04bb5b10e
--- /dev/null
+++ b/collector/webui/tables.go
@@ -0,0 +1,47 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package webui
+
+import (
+	"github.com/kataras/iris/v12"
+)
+
+func tablesHandler(ctx iris.Context) {
+	// metaClient := client.NewMetaClient(viper.GetString("meta_server"))
+	// tables, err := metaClient.ListTables()
+	// if err != nil {
+	// 	ctx.ResponseWriter().WriteString("Failed to list tables from MetaServer")
+	// 	ctx.StatusCode(iris.StatusInternalServerError)
+	// 	return
+	// }
+
+	// type tableHTMLRow struct {
+	// 	TableName string
+	// 	Link      string
+	// }
+	// var Tables []tableHTMLRow
+	// for _, tb := range tables {
+	// 	Tables = append(Tables, tableHTMLRow{TableName: tb.TableName})
+	// }
+	// ctx.ViewData("Tables", Tables)
+
+	err := ctx.View("tables.html")
+	if err != nil {
+		return
+	}
+}
diff --git a/collector/webui/webserver.go b/collector/webui/webserver.go
new file mode 100644
index 000000000..8c13dda9c
--- /dev/null
+++ b/collector/webui/webserver.go
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package webui
+
+import (
+	"context"
+	"time"
+
+	"github.com/kataras/iris/v12"
+	"github.com/prometheus/client_golang/prometheus/promhttp"
+)
+
+// StartWebServer starts an iris-powered HTTP server.
+func StartWebServer() {
+	app := iris.New()
+	app.Get("/", indexHandler)
+	app.Get("/tables", tablesHandler)
+	app.Get("/metrics", func(ctx iris.Context) {
+		handler := promhttp.Handler()
+		handler.ServeHTTP(ctx.ResponseWriter(), ctx.Request())
+	})
+
+	iris.RegisterOnInterrupt(func() {
+		// gracefully shutdown on interrupt
+		timeout := 5 * time.Second
+		ctx, cancel := context.WithTimeout(context.Background(), timeout)
+		defer cancel()
+		err := app.Shutdown(ctx)
+		if err != nil {
+			return
+		}
+	})
+
+	// Register the view engine to the views,
+	// this will load the templates.
+	tmpl := iris.HTML("./templates", ".html")
+	tmpl.Reload(true)
+	app.RegisterView(tmpl)
+
+	go func() {
+		err := app.Listen(":8080")
+		if err != nil {
+			return
+		}
+	}()
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pegasus.apache.org
For additional commands, e-mail: commits-help@pegasus.apache.org