From 84ebbb8ffed9f9d1cbf941daf9b236cb9978f6ea Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Sat, 11 Jan 2025 16:11:07 +0800 Subject: [PATCH 1/2] enh:Add TDengine benchmark support and fix docs --- Makefile | 6 +- README.md | 8 +- cmd/tsbs_generate_data/main.go | 1 + .../databases/tdengine/common.go | 59 ++ .../databases/tdengine/devops.go | 174 ++++ .../databases/tdengine/iot.go | 193 ++++ cmd/tsbs_load/parse_config.go | 6 +- cmd/tsbs_load_cratedb/creator.go | 13 +- cmd/tsbs_load_tdengine/main.go | 89 ++ cmd/tsbs_load_tdenginestmt2/main.go | 96 ++ cmd/tsbs_run_queries_tdengine/main.go | 122 +++ docs/tdengine.md | 143 +++ go.mod | 50 +- go.sum | 214 ++-- internal/inputs/generator_data.go | 11 +- pkg/data/serialize/point_serializer.go | 9 +- pkg/query/factories/init_factories.go | 2 + pkg/query/stats.go | 50 +- pkg/query/tdengine.go | 59 ++ pkg/targets/constants/constants.go | 4 + .../initializers/target_initializers.go | 9 +- pkg/targets/targets.go | 5 + pkg/targets/tdengine/async/handlerpool.go | 120 +++ pkg/targets/tdengine/async/row.go | 133 +++ pkg/targets/tdengine/benchmark.go | 66 ++ pkg/targets/tdengine/commonpool/pool.go | 133 +++ pkg/targets/tdengine/creator.go | 68 ++ pkg/targets/tdengine/file_data_source.go | 65 ++ pkg/targets/tdengine/implemented_target.go | 58 ++ pkg/targets/tdengine/process.go | 225 +++++ pkg/targets/tdengine/program_options.go | 15 + pkg/targets/tdengine/scan.go | 95 ++ pkg/targets/tdengine/serializer.go | 232 +++++ pkg/targets/tdengine/thread/locker.go | 33 + pkg/targets/tdengine/thread/locker_test.go | 41 + pkg/targets/tdenginestmt2/benchmark.go | 105 ++ pkg/targets/tdenginestmt2/cb.go | 23 + pkg/targets/tdenginestmt2/creator.go | 36 + pkg/targets/tdenginestmt2/file_data_source.go | 198 ++++ .../tdenginestmt2/implemented_target.go | 44 + pkg/targets/tdenginestmt2/process.go | 932 ++++++++++++++++++ pkg/targets/tdenginestmt2/program_options.go | 5 + pkg/targets/tdenginestmt2/scan.go | 207 ++++ pkg/targets/tdenginestmt2/serializer.go | 558 +++++++++++ pkg/targets/timescaledb/creator.go | 1 + .../full_cycle_minitest_tdengine.sh | 25 + scripts/load/load_tdengine.sh | 42 + scripts/load/load_tdenginestmt2.sh | 42 + scripts/run_queries/run_queries_tdengine.sh | 62 ++ 49 files changed, 4799 insertions(+), 88 deletions(-) create mode 100644 cmd/tsbs_generate_queries/databases/tdengine/common.go create mode 100644 cmd/tsbs_generate_queries/databases/tdengine/devops.go create mode 100644 cmd/tsbs_generate_queries/databases/tdengine/iot.go create mode 100644 cmd/tsbs_load_tdengine/main.go create mode 100644 cmd/tsbs_load_tdenginestmt2/main.go create mode 100644 cmd/tsbs_run_queries_tdengine/main.go create mode 100644 docs/tdengine.md create mode 100644 pkg/query/tdengine.go create mode 100644 pkg/targets/tdengine/async/handlerpool.go create mode 100644 pkg/targets/tdengine/async/row.go create mode 100644 pkg/targets/tdengine/benchmark.go create mode 100644 pkg/targets/tdengine/commonpool/pool.go create mode 100644 pkg/targets/tdengine/creator.go create mode 100644 pkg/targets/tdengine/file_data_source.go create mode 100644 pkg/targets/tdengine/implemented_target.go create mode 100644 pkg/targets/tdengine/process.go create mode 100644 pkg/targets/tdengine/program_options.go create mode 100644 pkg/targets/tdengine/scan.go create mode 100644 pkg/targets/tdengine/serializer.go create mode 100644 pkg/targets/tdengine/thread/locker.go create mode 100644 pkg/targets/tdengine/thread/locker_test.go create mode 100644 pkg/targets/tdenginestmt2/benchmark.go create mode 100644 pkg/targets/tdenginestmt2/cb.go create mode 100644 pkg/targets/tdenginestmt2/creator.go create mode 100644 pkg/targets/tdenginestmt2/file_data_source.go create mode 100644 pkg/targets/tdenginestmt2/implemented_target.go create mode 100644 pkg/targets/tdenginestmt2/process.go create mode 100644 pkg/targets/tdenginestmt2/program_options.go create mode 100644 pkg/targets/tdenginestmt2/scan.go create mode 100644 pkg/targets/tdenginestmt2/serializer.go create mode 100755 scripts/full_cycle_minitest/full_cycle_minitest_tdengine.sh create mode 100755 scripts/load/load_tdengine.sh create mode 100755 scripts/load/load_tdenginestmt2.sh create mode 100755 scripts/run_queries/run_queries_tdengine.sh diff --git a/Makefile b/Makefile index f6a73c0c3..0a7825dab 100644 --- a/Makefile +++ b/Makefile @@ -26,7 +26,8 @@ loaders: tsbs_load \ tsbs_load_siridb \ tsbs_load_timescaledb \ tsbs_load_victoriametrics \ - tsbs_load_questdb + tsbs_load_questdb \ + tsbs_load_tdengine runners: tsbs_run_queries_akumuli \ tsbs_run_queries_cassandra \ @@ -38,7 +39,8 @@ runners: tsbs_run_queries_akumuli \ tsbs_run_queries_timescaledb \ tsbs_run_queries_timestream \ tsbs_run_queries_victoriametrics \ - tsbs_run_queries_questdb + tsbs_run_queries_questdb \ + tsbs_run_queries_tdengine test: $(GOTEST) -v ./... diff --git a/README.md b/README.md index bf78a77df..385dfcd53 100644 --- a/README.md +++ b/README.md @@ -17,6 +17,7 @@ Current databases supported: + TimescaleDB [(supplemental docs)](docs/timescaledb.md) + Timestream [(supplemental docs)](docs/timestream.md) + VictoriaMetrics [(supplemental docs)](docs/victoriametrics.md) ++ TDengine [(supplemental docs)](docs/tdengine.md) ## Overview @@ -81,6 +82,7 @@ cases are implemented for each database: |TimescaleDB|X|X| |Timestream|X|| |VictoriaMetrics|X²|| +|TDengine|X|X| ¹ Does not support the `groupby-orderby-limit` query ² Does not support the `groupby-orderby-limit`, `lastpoint`, `high-cpu-1`, `high-cpu-all` queries @@ -112,6 +114,10 @@ $ cd $GOPATH/src/github.com/timescale/tsbs $ make ``` +**`Notice`** + +The make process will failed if the TDengine client isn't installed. Please refer to the [supplementary documentation for TDengine](docs/tdengine.md). + ## How to use TSBS Using TSBS for benchmarking involves 3 phases: data and query @@ -135,7 +141,7 @@ Variables needed: 1. how much time should be between each reading per device, in seconds. E.g., `10s` 1. and which database(s) you want to generate for. E.g., `timescaledb` (choose from `cassandra`, `clickhouse`, `cratedb`, `influx`, `mongo`, `questdb`, `siridb`, - `timescaledb` or `victoriametrics`) + `timescaledb` `victoriametrics` or `tdengine`) Given the above steps you can now generate a dataset (or multiple datasets, if you chose to generate for multiple databases) that can diff --git a/cmd/tsbs_generate_data/main.go b/cmd/tsbs_generate_data/main.go index de9deb540..d604d80dd 100644 --- a/cmd/tsbs_generate_data/main.go +++ b/cmd/tsbs_generate_data/main.go @@ -7,6 +7,7 @@ // MongoDB BSON format // TimescaleDB pseudo-CSV format (the same as for ClickHouse) // VictoriaMetrics bulk load format (the same as for InfluxDB) +// TDengine pseudo-CSV format // Supported use cases: // devops: scale is the number of hosts to simulate, with log messages diff --git a/cmd/tsbs_generate_queries/databases/tdengine/common.go b/cmd/tsbs_generate_queries/databases/tdengine/common.go new file mode 100644 index 000000000..bc1c8e0ff --- /dev/null +++ b/cmd/tsbs_generate_queries/databases/tdengine/common.go @@ -0,0 +1,59 @@ +package tdengine + +import ( + "time" + + "github.com/timescale/tsbs/cmd/tsbs_generate_queries/uses/devops" + "github.com/timescale/tsbs/cmd/tsbs_generate_queries/uses/iot" + "github.com/timescale/tsbs/cmd/tsbs_generate_queries/utils" + "github.com/timescale/tsbs/pkg/query" +) + +// BaseGenerator contains settings specific for Influx database. +type BaseGenerator struct { +} + +func (g *BaseGenerator) GenerateEmptyQuery() query.Query { + return query.NewTDengine() +} + +// fillInQuery fills the query struct with data. +func (g *BaseGenerator) fillInQuery(qi query.Query, humanLabel, humanDesc, table, sql string) { + q := qi.(*query.TDengine) + q.HumanLabel = []byte(humanLabel) + q.HumanDescription = []byte(humanDesc) + q.Hypertable = []byte(table) + q.SqlQuery = []byte(sql) +} + +// NewDevops creates a new devops use case query generator. +func (g *BaseGenerator) NewDevops(start, end time.Time, scale int) (utils.QueryGenerator, error) { + core, err := devops.NewCore(start, end, scale) + + if err != nil { + return nil, err + } + + devops := &Devops{ + BaseGenerator: g, + Core: core, + } + + return devops, nil +} + +// NewIoT creates a new iot use case query generator. +func (g *BaseGenerator) NewIoT(start, end time.Time, scale int) (utils.QueryGenerator, error) { + core, err := iot.NewCore(start, end, scale) + + if err != nil { + return nil, err + } + + iot := &IoT{ + BaseGenerator: g, + Core: core, + } + + return iot, nil +} diff --git a/cmd/tsbs_generate_queries/databases/tdengine/devops.go b/cmd/tsbs_generate_queries/databases/tdengine/devops.go new file mode 100644 index 000000000..814d8880c --- /dev/null +++ b/cmd/tsbs_generate_queries/databases/tdengine/devops.go @@ -0,0 +1,174 @@ +package tdengine + +import ( + "fmt" + "strings" + "time" + + "github.com/timescale/tsbs/cmd/tsbs_generate_queries/uses/devops" + "github.com/timescale/tsbs/pkg/query" +) + +// TODO: Remove the need for this by continuing to bubble up errors +func panicIfErr(err error) { + if err != nil { + panic(err.Error()) + } +} + +// Devops produces TimescaleDB-specific queries for all the devops query types. +type Devops struct { + *BaseGenerator + *devops.Core +} + +// getHostWhereWithHostnames creates WHERE SQL statement for multiple hostnames. +// NOTE 'WHERE' itself is not included, just hostname filter clauses, ready to concatenate to 'WHERE' string +func (d *Devops) getHostWhereWithHostnames(hostnames []string) string { + var hostnameClauses []string + for _, s := range hostnames { + hostnameClauses = append(hostnameClauses, fmt.Sprintf("'%s'", s)) + } + return fmt.Sprintf("tbname IN (%s)", strings.Join(hostnameClauses, ",")) +} + +// getHostWhereString gets multiple random hostnames and creates a WHERE SQL statement for these hostnames. +func (d *Devops) getHostWhereString(nHosts int) string { + hostnames, err := d.GetRandomHosts(nHosts) + panicIfErr(err) + return d.getHostWhereWithHostnames(hostnames) +} + +func (d *Devops) getSelectClausesAggMetrics(agg string, metrics []string) []string { + selectClauses := make([]string, len(metrics)) + for i, m := range metrics { + selectClauses[i] = fmt.Sprintf("%s(%s)", agg, m) + } + + return selectClauses +} + +func (d *Devops) GroupByTime(qi query.Query, nHosts, numMetrics int, timeRange time.Duration) { + interval := d.Interval.MustRandWindow(timeRange) + metrics, err := devops.GetCPUMetricsSlice(numMetrics) + panicIfErr(err) + selectClauses := d.getSelectClausesAggMetrics("max", metrics) + if len(selectClauses) < 1 { + panic(fmt.Sprintf("invalid number of select clauses: got %d", len(selectClauses))) + } + + //SELECT _wstart as ts,max(usage_user) FROM cpu WHERE tbname IN ('host_249') AND ts >= 1451618560000 AND ts < 1451622160000 INTERVAL(1m) ; + //SELECT _wstart as ts,max(usage_user) FROM host_249 WHERE ts >= 1451618560000 AND ts < 1451622160000 INTERVAL(1m) ; + sql := "" + if nHosts == 1 { + hostnames, err := d.GetRandomHosts(nHosts) + panicIfErr(err) + sql = fmt.Sprintf(`SELECT _wstart as ts,%s FROM %s WHERE ts >= %d AND ts < %d INTERVAL(1m)`, + strings.Join(selectClauses, ", "), + hostnames[0], + interval.StartUnixMillis(), + interval.EndUnixMillis()) + + } else { + sql = fmt.Sprintf(`SELECT _wstart as ts,%s FROM cpu WHERE %s AND ts >= %d AND ts < %d INTERVAL(1m)`, + strings.Join(selectClauses, ", "), + d.getHostWhereString(nHosts), + interval.StartUnixMillis(), + interval.EndUnixMillis()) + } + + humanLabel := fmt.Sprintf("TDengine %d cpu metric(s), random %4d hosts, random %s by 1m", numMetrics, nHosts, timeRange) + humanDesc := fmt.Sprintf("%s: %s", humanLabel, interval.StartString()) + d.fillInQuery(qi, humanLabel, humanDesc, devops.TableName, sql) +} + +func (d *Devops) GroupByOrderByLimit(qi query.Query) { + interval := d.Interval.MustRandWindow(time.Hour) + //SELECT _wstart as ts,max(usage_user) FROM cpu WHERE ts < 1451618228646 INTERVAL(1m) LIMIT 5; + sql := fmt.Sprintf(`SELECT _wstart as ts,max(usage_user) FROM cpu WHERE ts < %d INTERVAL(1m) LIMIT 5`, + interval.EndUnixMillis()) + + humanLabel := "TDengine max cpu over last 5 min-intervals (random end)" + humanDesc := fmt.Sprintf("%s: %s", humanLabel, interval.EndString()) + d.fillInQuery(qi, humanLabel, humanDesc, devops.TableName, sql) +} + +// GroupByTimeAndPrimaryTag selects the AVG of numMetrics metrics under 'cpu' per device per hour for a day, +func (d *Devops) GroupByTimeAndPrimaryTag(qi query.Query, numMetrics int) { + metrics, err := devops.GetCPUMetricsSlice(numMetrics) + panicIfErr(err) + interval := d.Interval.MustRandWindow(devops.DoubleGroupByDuration) + + selectClauses := d.getSelectClausesAggMetrics("avg", metrics) + //SELECT _wstart as ts,tbname, avg(usage_user) from cpu where ts >= 1451733760646 and ts < 1451776960646 partition by tbname interval(1h) order by tbname,ts asc; + //SELECT _wstart as ts,tbname, avg(usage_user), avg(usage_system), avg(usage_idle), avg(usage_nice), avg(usage_iowait), avg(usage_irq), avg(usage_softirq), avg(usage_steal), avg(usage_guest), avg(usage_guest_nice) from cpu where ts >= 1451733760646 and ts < 1451776960646 partition by tbname interval(1h) order by tbname,ts asc; + sql := fmt.Sprintf("SELECT _wstart as ts,tbname,%s from cpu where ts >= %d and ts < %d partition by tbname interval(1h) order by tbname,ts asc", strings.Join(selectClauses, ", "), interval.StartUnixMillis(), interval.EndUnixMillis()) + + humanLabel := devops.GetDoubleGroupByLabel("TDengine", numMetrics) + humanDesc := fmt.Sprintf("%s: %s", humanLabel, interval.StartString()) + d.fillInQuery(qi, humanLabel, humanDesc, devops.TableName, sql) +} + +func (d *Devops) MaxAllCPU(qi query.Query, nHosts int, duration time.Duration) { + interval := d.Interval.MustRandWindow(duration) + + metrics := devops.GetAllCPUMetrics() + selectClauses := d.getSelectClausesAggMetrics("max", metrics) + //SELECT _wstart as ts,max(usage_user), max(usage_system), max(usage_idle), max(usage_nice), max(usage_iowait), max(usage_irq), max(usage_softirq), max(usage_steal), max(usage_guest), max(usage_guest_nice) FROM host_249 WHERE ts >= 1451648911646 AND ts < 1451677711646 interval(1h); + //SELECT_wstart as ts, max(usage_user), max(usage_system), max(usage_idle), max(usage_nice), max(usage_iowait), max(usage_irq), max(usage_softirq), max(usage_steal), max(usage_guest), max(usage_guest_nice) FROM cpu WHERE tbname IN ('host_249','host_403','host_435','host_39','host_139','host_75','host_315','host_121') AND ts >= 1451648911646 AND ts < 1451677711646 interval(1h) + + sql := "" + if nHosts == 1 { + hostnames, err := d.GetRandomHosts(nHosts) + panicIfErr(err) + sql = fmt.Sprintf(`SELECT _wstart as ts,%s FROM %s WHERE ts >= %d AND ts < %d interval(1h)`, + strings.Join(selectClauses, ", "), + hostnames[0], + interval.StartUnixMillis(), + interval.EndUnixMillis()) + } else { + sql = fmt.Sprintf(`SELECT _wstart as ts,%s FROM cpu WHERE %s AND ts >= %d AND ts < %d interval(1h)`, + strings.Join(selectClauses, ", "), + d.getHostWhereString(nHosts), + interval.StartUnixMillis(), + interval.EndUnixMillis()) + } + humanLabel := devops.GetMaxAllLabel("TDengine", nHosts) + humanDesc := fmt.Sprintf("%s: %s", humanLabel, interval.StartString()) + d.fillInQuery(qi, humanLabel, humanDesc, devops.TableName, sql) +} + +func (d *Devops) LastPointPerHost(qi query.Query) { + //SELECT last_row(*),tbname from cpu group by tbname; + sql := "SELECT last_row(*),tbname from cpu group by tbname" + humanLabel := "TDengine last row per host" + humanDesc := humanLabel + d.fillInQuery(qi, humanLabel, humanDesc, devops.TableName, sql) +} + +func (d *Devops) HighCPUForHosts(qi query.Query, nHosts int) { + interval := d.Interval.MustRandWindow(devops.HighCPUDuration) + var hostWhereClause string + if nHosts == 0 { + hostWhereClause = "" + } else { + hostWhereClause = fmt.Sprintf("AND %s", d.getHostWhereString(nHosts)) + } + //SELECT ts,usage_user,usage_system,usage_idle,usage_nice,usage_iowait,usage_irq,usage_softirq,usage_steal,usage_guest,usage_guest_nice FROM cpu WHERE usage_user > 90.0 and ts >= 1451777731138 AND ts < 1451820931138 AND tbname IN ('host_35') + //modify:SELECT * FROM host_35 WHERE usage_user > 90.0 and ts >= 1451777731138 AND ts < 1451820931138 + + sql := "" + if nHosts == 1 { + hostnames, err := d.GetRandomHosts(nHosts) + panicIfErr(err) + sql = fmt.Sprintf(`SELECT * FROM %s WHERE usage_user > 90.0 and ts >= %d AND ts < %d `, + hostnames[0], interval.StartUnixMillis(), interval.EndUnixMillis()) + } else { + sql = fmt.Sprintf(`SELECT ts,usage_user,usage_system,usage_idle,usage_nice,usage_iowait,usage_irq,usage_softirq,usage_steal,usage_guest,usage_guest_nice FROM cpu WHERE usage_user > 90.0 and ts >= %d AND ts < %d %s`, + interval.StartUnixMillis(), interval.EndUnixMillis(), hostWhereClause) + } + humanLabel, err := devops.GetHighCPULabel("TDengine", nHosts) + panicIfErr(err) + humanDesc := fmt.Sprintf("%s: %s", humanLabel, interval.StartString()) + d.fillInQuery(qi, humanLabel, humanDesc, devops.TableName, sql) +} diff --git a/cmd/tsbs_generate_queries/databases/tdengine/iot.go b/cmd/tsbs_generate_queries/databases/tdengine/iot.go new file mode 100644 index 000000000..bf8267b77 --- /dev/null +++ b/cmd/tsbs_generate_queries/databases/tdengine/iot.go @@ -0,0 +1,193 @@ +package tdengine + +import ( + "fmt" + "strings" + "time" + + "github.com/timescale/tsbs/cmd/tsbs_generate_queries/uses/iot" + "github.com/timescale/tsbs/pkg/query" +) + +// IoT produces TDengine-specific queries for all the iot query types. +type IoT struct { + *iot.Core + *BaseGenerator +} + +//last-loc +//single-last-loc +//low-fuel +//avg-vs-projected-fuel-consumption +//avg-daily-driving-duration +//daily-activity + +func (i *IoT) getTrucksWhereWithNames(names []string) string { + var nameClauses []string + + for _, s := range names { + nameClauses = append(nameClauses, fmt.Sprintf("'%s'", s)) + } + return fmt.Sprintf("name IN (%s)", strings.Join(nameClauses, ",")) +} + +// getHostWhereString gets multiple random hostnames and creates a WHERE SQL statement for these hostnames. +func (i *IoT) getTruckWhereString(nTrucks int) string { + names, err := i.GetRandomTrucks(nTrucks) + panicIfErr(err) + return i.getTrucksWhereWithNames(names) +} + +// LastLocByTruck finds the truck location for nTrucks. +func (i *IoT) LastLocByTruck(qi query.Query, nTrucks int) { + sql := fmt.Sprintf(`SELECT last_row(ts),last_row(latitude),last_row(longitude) FROM readings WHERE %s GROUP BY name`, + i.getTruckWhereString(nTrucks)) + + humanLabel := "TDengine last location by specific truck" + humanDesc := fmt.Sprintf("%s: random %4d trucks", humanLabel, nTrucks) + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// LastLocPerTruck finds all the truck locations along with truck and driver names. +func (i *IoT) LastLocPerTruck(qi query.Query) { + //SELECT last_row(ts),name,driver,latitude,longitude FROM readings WHERE fleet='South' and name IS NOT NULL partition BY name,driver; + sql := fmt.Sprintf(`SELECT last_row(ts),name,driver,latitude,longitude FROM readings WHERE fleet='%s' and name IS NOT NULL partition BY name,driver`, + i.GetRandomFleet()) + + humanLabel := "TDengine last location per truck" + humanDesc := humanLabel + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// TrucksWithLowFuel finds all trucks with low fuel (less than 10%). +func (i *IoT) TrucksWithLowFuel(qi query.Query) { + //SELECT last_row(ts),name,driver,fuel_state FROM diagnostics WHERE fuel_state <= 0.1 AND fleet = 'South' and name IS NOT NULL GROUP BY name,driver ; + sql := fmt.Sprintf(`SELECT last_row(ts),name,driver,fuel_state FROM diagnostics WHERE fuel_state <= 0.1 AND fleet = '%s' and name IS NOT NULL GROUP BY name,driver`, + i.GetRandomFleet()) + + humanLabel := "TDengine trucks with low fuel" + humanDesc := fmt.Sprintf("%s: under 10 percent", humanLabel) + + i.fillInQuery(qi, humanLabel, humanDesc, iot.DiagnosticsTableName, sql) +} + +// TrucksWithHighLoad finds all trucks that have load over 90%. +func (i *IoT) TrucksWithHighLoad(qi query.Query) { + //SELECT ts,name,driver,current_load,load_capacity FROM (SELECT last_row(ts) as ts,name,driver, current_load,load_capacity FROM diagnostics WHERE fleet = 'South' partition by name,driver) WHERE current_load>= (0.9 * load_capacity); + //pre sql := fmt.Sprintf("SELECT ts,name,driver,current_load,load_capacity FROM (SELECT last_row(ts) as ts,name,driver, current_load,load_capacity FROM diagnostics WHERE fleet = '%s' partition by name,driver) WHERE current_load>= (0.9 * load_capacity)", i.GetRandomFleet()) + sql := fmt.Sprintf("SELECT ts,name,driver,current_load,load_capacity FROM (SELECT last_row(ts) as ts,name,driver, current_load,load_capacity FROM diagnostics WHERE fleet = '%s' partition by name,driver) WHERE current_load>= (0.9 * load_capacity)", i.GetRandomFleet()) + + humanLabel := "TDengine trucks with high load" + humanDesc := fmt.Sprintf("%s: over 90 percent", humanLabel) + + i.fillInQuery(qi, humanLabel, humanDesc, iot.DiagnosticsTableName, sql) +} + +// StationaryTrucks finds all trucks that have low average velocity in a time window. +func (i *IoT) StationaryTrucks(qi query.Query) { + interval := i.Interval.MustRandWindow(iot.StationaryDuration) + //select name,driver from (SELECT name,driver,fleet ,avg(velocity) as mean_velocity FROM readings WHERE ts > '2016-01-01T15:07:21Z' AND ts <= '2016-01-01T16:17:21Z' partition BY name,driver,fleet interval(10m) LIMIT 1) WHERE fleet = 'West' AND mean_velocity < 1 ; + sql := fmt.Sprintf("select name,driver from (SELECT name,driver,avg(velocity) as mean_velocity FROM readings WHERE ts > %d AND ts <= %d and fleet = '%s' partition BY name,driver interval(10m) LIMIT 1) WHERE mean_velocity < 1;", interval.StartUnixMillis(), interval.EndUnixMillis(), i.GetRandomFleet()) + //sql := fmt.Sprintf("SELECT name,driver FROM readings WHERE ts > %d AND ts <= %d and fleet = '%s' partition BY name,driver,fleet interval(10m) having (avg(velocity) < 1) LIMIT 1;", interval.StartUnixMillis(), interval.EndUnixMillis(), i.GetRandomFleet()) + humanLabel := "TDengine stationary trucks" + humanDesc := fmt.Sprintf("%s: with low avg velocity in last 10 minutes", humanLabel) + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// TrucksWithLongDrivingSessions finds all trucks that have not stopped at least 20 mins in the last 4 hours. +func (i *IoT) TrucksWithLongDrivingSessions(qi query.Query) { + interval := i.Interval.MustRandWindow(iot.LongDrivingSessionDuration) + //SELECT name,driver FROM(SELECT _wstart as ts,name,driver,avg(velocity) as mean_velocity FROM readings WHERE fleet ="West" AND ts > '2016-01-03T13:46:34Z' AND ts <= '2016-01-03T17:46:34Z' partition BY name,driver interval(10m)) WHERE mean_velocity > 1 GROUP BY name,driver having count(*) > 22 + //pre sql := fmt.Sprintf("SELECT name,driver FROM(SELECT _wstart as ts,name,driver,avg(velocity) as mean_velocity FROM readings WHERE fleet =\"%s\" AND ts > %d AND ts <= %d partition BY name,driver interval(10m)) WHERE mean_velocity > 1 GROUP BY name,driver having count(*) > %d", i.GetRandomFleet(), interval.StartUnixMillis(), interval.EndUnixMillis(), tenMinutePeriods(5, iot.LongDrivingSessionDuration)) + sql := fmt.Sprintf("SELECT name,driver FROM(SELECT name,driver,avg(velocity) as mean_velocity FROM readings WHERE fleet =\"%s\" AND ts > %d AND ts <= %d partition BY name,driver interval(10m)) WHERE mean_velocity > 1 GROUP BY name,driver having count(*) > %d", i.GetRandomFleet(), interval.StartUnixMillis(), interval.EndUnixMillis(), tenMinutePeriods(5, iot.LongDrivingSessionDuration)) + humanLabel := "TDengine trucks with longer driving sessions" + humanDesc := fmt.Sprintf("%s: stopped less than 20 mins in 4 hour period", humanLabel) + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// TrucksWithLongDailySessions finds all trucks that have driven more than 10 hours in the last 24 hours. +func (i *IoT) TrucksWithLongDailySessions(qi query.Query) { + //SELECT name,driver FROM(SELECT name,driver,avg(velocity) as mean_velocity FROM readings WHERE fleet ='West' AND ts > '2016-01-01T12:31:37Z' AND ts <= '2016-01-05T12:31:37Z' partition BY name,driver interval(10m) ) WHERE mean_velocity > 1 GROUP BY name,driver having count(*) > 60 + + interval := i.Interval.MustRandWindow(iot.DailyDrivingDuration) + sql := fmt.Sprintf("SELECT name,driver FROM(SELECT name,driver,avg(velocity) as mean_velocity FROM readings WHERE fleet ='%s' AND ts > %d AND ts <= %d partition BY name,driver interval(10m) ) WHERE mean_velocity > 1 GROUP BY name,driver having count(*) > %d", i.GetRandomFleet(), interval.StartUnixMillis(), interval.EndUnixMillis(), tenMinutePeriods(35, iot.DailyDrivingDuration)) + + humanLabel := "TDengine trucks with longer daily sessions" + humanDesc := fmt.Sprintf("%s: drove more than 10 hours in the last 24 hours", humanLabel) + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// AvgVsProjectedFuelConsumption calculates average and projected fuel consumption per fleet. +func (i *IoT) AvgVsProjectedFuelConsumption(qi query.Query) { + //select avg(fuel_consumption) as avg_fuel_consumption,avg(nominal_fuel_consumption) as nominal_fuel_consumption from readings where velocity > 1 group by fleet + sql := fmt.Sprintf("select avg(fuel_consumption) as avg_fuel_consumption,avg(nominal_fuel_consumption) as nominal_fuel_consumption from readings where velocity > 1 group by fleet") + humanLabel := "TDengine average vs projected fuel consumption per fleet" + humanDesc := humanLabel + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// AvgDailyDrivingDuration finds the average driving duration per driver. +func (i *IoT) AvgDailyDrivingDuration(qi query.Query) { + //select fleet,name,driver,avg(hours_driven) as avg_daily_hours from( select _wstart as ts,fleet,name,driver,count(mv)/6 as hours_driven from ( select _wstart as ts,fleet,tbname,name,driver,avg(velocity) as mv from readings where ts > '2016-01-01T00:00:00Z' and ts < '2016-01-05T00:00:01Z' partition by fleet,tbname,name,driver interval(10m) ) where mv >1 partition by fleet,name,driver interval(1d) )partition by fleet,name,driver ; + sql := fmt.Sprintf("select fleet,name,driver,avg(hours_driven) as avg_daily_hours from( select _wstart as ts,fleet,name,driver,count(mv)/6 as hours_driven from ( select _wstart as ts,fleet,tbname,name,driver,avg(velocity) as mv from readings where ts > %d and ts < %d partition by fleet,tbname,name,driver interval(10m) ) where mv >1 partition by fleet,name,driver interval(1d) )partition by fleet,name,driver ;", i.Interval.StartUnixMillis(), i.Interval.EndUnixMillis()) + + humanLabel := "TDengine average driver driving duration per day" + humanDesc := humanLabel + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// AvgDailyDrivingSession finds the average driving session without stopping per driver per day. +func (i *IoT) AvgDailyDrivingSession(qi query.Query) { + // select _wstart as ts,name,avg(ela) from (select ts,name,ela from (SELECT ts,name, diff(difka) as dif, diff(cast(ts as bigint)) as ela FROM (SELECT ts,name,difka FROM (SELECT ts,name,diff(mv) AS difka FROM (SELECT _wstart as ts,name,cast(cast(floor(avg(velocity)/5) as bool) as int) AS mv FROM readings WHERE name is not null AND ts > 1451637149138 AND ts < 1451637749138 partition by name interval(10m))partition BY name ) WHERE difka!=0 order by ts) partition BY name order by ts ) WHERE dif = -2 partition BY name order by ts ) partition BY name interval(1d); + interval := i.Interval + sql := fmt.Sprintf(" select _wstart as ts,name,avg(ela) from (select ts,name,ela from (SELECT ts,name, diff(difka) as dif, diff(cast(ts as bigint)) as ela FROM (SELECT ts,name,difka FROM (SELECT ts,name,diff(mv) AS difka FROM (SELECT _wstart as ts,name,cast(cast(floor(avg(velocity)/5) as bool) as int) AS mv FROM readings WHERE name is not null AND ts > %d AND ts < %d partition by name interval(10m))partition BY name ) WHERE difka!=0 order by ts) partition BY name order by ts ) WHERE dif = -2 partition BY name order by ts ) partition BY name interval(1d);", interval.StartUnixMillis(), interval.EndUnixMillis()) + humanLabel := "TDengine average driver driving session without stopping per day" + humanDesc := humanLabel + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// AvgLoad finds the average load per truck model per fleet. +func (i *IoT) AvgLoad(qi query.Query) { + //select fleet,model,load_capacity,avg(ml/load_capacity) from(SELECT fleet, model,tbname,load_capacity ,avg(current_load) AS ml FROM diagnostics where name is not null partition BY fleet, model,tbname,load_capacity) partition BY fleet, model,load_capacity; + sql := fmt.Sprintf("select fleet,model,load_capacity,avg(ml/load_capacity) from(SELECT fleet, model,tbname,load_capacity ,avg(current_load) AS ml FROM diagnostics where name is not null partition BY fleet, model,tbname,load_capacity) partition BY fleet, model,load_capacity") + + humanLabel := "TDengine average load per truck model per fleet" + humanDesc := humanLabel + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// DailyTruckActivity returns the number of hours trucks has been active (not out-of-commission) per day per fleet per model. +func (i *IoT) DailyTruckActivity(qi query.Query) { + //SELECT _wstart as ts,model,fleet,count(ms1)/144 FROM (SELECT _wstart as ts1,model, fleet,avg(status) AS ms1 FROM diagnostics WHERE ts >= 1451606400000 AND ts < 1451952001000 partition by model, fleet, tbname interval(10m)) WHERE ms1<1 partition by model, fleet interval(1d) + sql := fmt.Sprintf("SELECT _wstart as ts,model,fleet,count(ms1)/144 FROM (SELECT _wstart as ts1,model, fleet,avg(status) AS ms1 FROM diagnostics WHERE ts >= %d AND ts < %d partition by model, fleet, tbname interval(10m)) WHERE ms1<1 partition by model, fleet interval(1d)", i.Interval.StartUnixMillis(), i.Interval.EndUnixMillis()) + humanLabel := "TDengine daily truck activity per fleet per model" + humanDesc := humanLabel + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +// TruckBreakdownFrequency calculates the amount of times a truck model broke down in the last period. +func (i *IoT) TruckBreakdownFrequency(qi query.Query) { + // SELECT model,count(state_changed) FROM (SELECT _rowts,model,diff(broken_down) AS state_changed FROM (SELECT model,tb,cast(cast(floor(2*(nzs)) as bool) as int) AS broken_down FROM (SELECT _wstart as ts,model,tbname as tb, sum(cast(cast(status as bool) as int))/count(cast(cast(status as bool) as int)) AS nzs FROM diagnostics WHERE ts >= '2016-01-01T00:00:00Z' AND ts < '2023-01-05T00:00:01Z' partition BY tbname,model interval(10m)) order by ts ) partition BY tb,model ) WHERE state_changed = 1 partition BY model ; + sql := fmt.Sprintf("SELECT model,count(state_changed) FROM (SELECT _rowts,model,diff(broken_down) AS state_changed FROM (SELECT ts,model,tb,cast(cast(floor(2*(nzs)) as bool) as int) AS broken_down FROM (SELECT _wstart as ts,model,tbname as tb, sum(cast(cast(status as bool) as int))/count(cast(cast(status as bool) as int)) AS nzs FROM diagnostics WHERE ts >= %d AND ts < %d partition BY tbname,model interval(10m)) order by ts) partition BY tb,model) WHERE state_changed = 1 partition BY model", i.Interval.StartUnixMillis(), i.Interval.EndUnixMillis()) + + humanLabel := "TDengine truck breakdown frequency per model" + humanDesc := humanLabel + + i.fillInQuery(qi, humanLabel, humanDesc, iot.ReadingsTableName, sql) +} + +func tenMinutePeriods(minutesPerHour float64, duration time.Duration) int { + durationMinutes := duration.Minutes() + leftover := minutesPerHour * duration.Hours() + return int((durationMinutes - leftover) / 10) +} diff --git a/cmd/tsbs_load/parse_config.go b/cmd/tsbs_load/parse_config.go index c45449684..f085986d3 100644 --- a/cmd/tsbs_load/parse_config.go +++ b/cmd/tsbs_load/parse_config.go @@ -3,6 +3,7 @@ package main import ( "errors" "fmt" + "github.com/blagojts/viper" "github.com/timescale/tsbs/load" "github.com/timescale/tsbs/pkg/data/source" @@ -47,7 +48,10 @@ func parseConfig(target targets.ImplementedTarget, v *viper.Viper) (targets.Benc if err != nil { return nil, nil, err } - + configurableBenchmark, ok := benchmark.(targets.ConfigurableBenchmark) + if ok { + configurableBenchmark.SetConfig(loaderConfigInternal.BatchSize, loaderConfig.Workers) + } return benchmark, load.GetBenchmarkRunner(*loaderConfigInternal), nil } diff --git a/cmd/tsbs_load_cratedb/creator.go b/cmd/tsbs_load_cratedb/creator.go index 914d86b92..f9fd65f46 100644 --- a/cmd/tsbs_load_cratedb/creator.go +++ b/cmd/tsbs_load_cratedb/creator.go @@ -58,18 +58,21 @@ func (d *dbCreator) Init() { // First N lines are the header, describing the data structure. // The first line contains the tags table name followed by a comma-separated // list of tags: -// tags,hostname,region,datacenter,rack,os,arch,team,service,service_version +// +// tags,hostname,region,datacenter,rack,os,arch,team,service,service_version // // The second through N-1 line containing table name (ex.: 'disk') followed by // list of column names, comma-separated: -// disk,total,free,used,used_percent,inodes_total,inodes_free,inodes_used +// +// disk,total,free,used,used_percent,inodes_total,inodes_free,inodes_used // // The last line being blank to separate the header from the data. // // Header example: -// tags,hostname,region,datacenter,rack,os,arch,team,service,service_version,service_environment -// disk,total,free,used,used_percent,inodes_total,inodes_free,inodes_used -// nginx,accepts,active,handled,reading,requests,waiting,writing +// +// tags,hostname,region,datacenter,rack,os,arch,team,service,service_version,service_environment +// disk,total,free,used,used_percent,inodes_total,inodes_free,inodes_used +// nginx,accepts,active,handled,reading,requests,waiting,writing func (d *dbCreator) readDataHeader(header *common.GeneratedDataHeaders) ([]*tableDef, error) { var tableDefs []*tableDef for tableName, fieldCols := range header.FieldKeys { diff --git a/cmd/tsbs_load_tdengine/main.go b/cmd/tsbs_load_tdengine/main.go new file mode 100644 index 000000000..2a9e79d37 --- /dev/null +++ b/cmd/tsbs_load_tdengine/main.go @@ -0,0 +1,89 @@ +package main + +import ( + "fmt" + "log" + "os" + "runtime/pprof" + + "github.com/blagojts/viper" + "github.com/spf13/pflag" + "github.com/timescale/tsbs/internal/utils" + "github.com/timescale/tsbs/load" + "github.com/timescale/tsbs/pkg/data/source" + "github.com/timescale/tsbs/pkg/targets/tdengine" +) + +func initProgramOptions() (*tdengine.LoadingOptions, load.BenchmarkRunner, *load.BenchmarkRunnerConfig) { + target := tdengine.NewTarget() + loaderConf := load.BenchmarkRunnerConfig{} + loaderConf.AddToFlagSet(pflag.CommandLine) + target.TargetSpecificFlags("", pflag.CommandLine) + pflag.Parse() + err := utils.SetupConfigFile() + + if err != nil { + panic(fmt.Errorf("fatal error config file: %s", err)) + } + + if err := viper.Unmarshal(&loaderConf); err != nil { + panic(fmt.Errorf("unable to decode config: %s", err)) + } + opts := tdengine.LoadingOptions{} + viper.SetTypeByDefaultValue(true) + opts.User = viper.GetString("user") + opts.Pass = viper.GetString("pass") + opts.Host = viper.GetString("host") + opts.Port = viper.GetInt("port") + vgroups := viper.GetInt("vgroups") + if vgroups > 0 { + opts.VGroups = vgroups + } + buffer := viper.GetInt("buffer") + if buffer > 0 { + opts.Buffer = buffer + } + pages := viper.GetInt("pages") + if pages > 0 { + opts.Pages = pages + } + sttTrigger := viper.GetInt("stt_trigger") + if sttTrigger > 0 { + opts.SttTrigger = sttTrigger + } + if viper.IsSet("wal_fsync_period") { + walFsyncPeriod := viper.GetInt("wal_fsync_period") + opts.WalFsyncPeriod = &walFsyncPeriod + } + if viper.IsSet("wal_level") { + walLevel := viper.GetInt("wal_level") + opts.WalLevel = &walLevel + } + if viper.IsSet("db_parameters") { + opts.DBParameters = viper.GetString("db_parameters") + } + loaderConf.HashWorkers = true + loaderConf.NoFlowControl = true + loaderConf.ChannelCapacity = 50 + loader := load.GetBenchmarkRunner(loaderConf) + return &opts, loader, &loaderConf +} +func main() { + f, err := os.Create("./cpu.prof") + if err != nil { + log.Fatal("could not create CPU profile: ", err) + } + if err := pprof.StartCPUProfile(f); err != nil { + log.Fatal("could not start CPU profile: ", err) + } + defer pprof.StopCPUProfile() + opts, loader, loaderConf := initProgramOptions() + benchmark, err := tdengine.NewBenchmark(loaderConf.DBName, opts, &source.DataSourceConfig{ + Type: source.FileDataSourceType, + File: &source.FileDataSourceConfig{Location: loaderConf.FileName}, + }) + if err != nil { + panic(err) + } + loader.RunBenchmark(benchmark) +} diff --git a/cmd/tsbs_load_tdenginestmt2/main.go b/cmd/tsbs_load_tdenginestmt2/main.go new file mode 100644 index 000000000..6db7ce176 --- /dev/null +++ b/cmd/tsbs_load_tdenginestmt2/main.go @@ -0,0 +1,96 @@ +package main + +import ( + "fmt" + "log" + _ "net/http/pprof" + "os" + "runtime/pprof" + + "github.com/blagojts/viper" + "github.com/spf13/pflag" + "github.com/timescale/tsbs/internal/utils" + "github.com/timescale/tsbs/load" + "github.com/timescale/tsbs/pkg/data/source" + "github.com/timescale/tsbs/pkg/targets" + "github.com/timescale/tsbs/pkg/targets/tdengine" + "github.com/timescale/tsbs/pkg/targets/tdenginestmt2" +) + +func initProgramOptions() (*tdengine.LoadingOptions, load.BenchmarkRunner, *load.BenchmarkRunnerConfig) { + target := tdengine.NewTarget() + loaderConf := load.BenchmarkRunnerConfig{} + loaderConf.AddToFlagSet(pflag.CommandLine) + target.TargetSpecificFlags("", pflag.CommandLine) + pflag.Parse() + err := utils.SetupConfigFile() + + if err != nil { + panic(fmt.Errorf("fatal error config file: %s", err)) + } + + if err := viper.Unmarshal(&loaderConf); err != nil { + panic(fmt.Errorf("unable to decode config: %s", err)) + } + opts := tdengine.LoadingOptions{} + viper.SetTypeByDefaultValue(true) + opts.User = viper.GetString("user") + opts.Pass = viper.GetString("pass") + opts.Host = viper.GetString("host") + opts.Port = viper.GetInt("port") + vgroups := viper.GetInt("vgroups") + if vgroups > 0 { + opts.VGroups = vgroups + } + buffer := viper.GetInt("buffer") + if buffer > 0 { + opts.Buffer = buffer + } + pages := viper.GetInt("pages") + if pages > 0 { + opts.Pages = pages + } + sttTrigger := viper.GetInt("stt_trigger") + if sttTrigger > 0 { + opts.SttTrigger = sttTrigger + } + if viper.IsSet("wal_fsync_period") { + walFsyncPeriod := viper.GetInt("wal_fsync_period") + opts.WalFsyncPeriod = &walFsyncPeriod + } + if viper.IsSet("wal_level") { + walLevel := viper.GetInt("wal_level") + opts.WalLevel = &walLevel + } + if viper.IsSet("db_parameters") { + opts.DBParameters = viper.GetString("db_parameters") + } + loaderConf.HashWorkers = true + loaderConf.NoFlowControl = true + loaderConf.ChannelCapacity = 50 + loader := load.GetBenchmarkRunner(loaderConf) + return &opts, loader, &loaderConf +} +func main() { + //go http.ListenAndServe(":3666", nil) + + f, err := os.Create("./cpu.prof") + if err != nil { + log.Fatal("could not create CPU profile: ", err) + } + if err := pprof.StartCPUProfile(f); err != nil { + log.Fatal("could not start CPU profile: ", err) + } + defer pprof.StopCPUProfile() + opts, loader, loaderConf := initProgramOptions() + + benchmark, err := tdenginestmt2.NewBenchmark(loaderConf.DBName, opts, &source.DataSourceConfig{ + Type: source.FileDataSourceType, + File: &source.FileDataSourceConfig{Location: loaderConf.FileName}, + }) + benchmark.(targets.ConfigurableBenchmark).SetConfig(loaderConf.BatchSize, loaderConf.Workers) + if err != nil { + panic(err) + } + loader.RunBenchmark(benchmark) +} diff --git a/cmd/tsbs_run_queries_tdengine/main.go b/cmd/tsbs_run_queries_tdengine/main.go new file mode 100644 index 000000000..39d6d0a46 --- /dev/null +++ b/cmd/tsbs_run_queries_tdengine/main.go @@ -0,0 +1,122 @@ +package main + +import ( + "database/sql/driver" + "fmt" + "strings" + "time" + + "github.com/blagojts/viper" + "github.com/spf13/pflag" + "github.com/timescale/tsbs/internal/utils" + "github.com/timescale/tsbs/pkg/query" + "github.com/timescale/tsbs/pkg/targets/tdengine/async" + "github.com/timescale/tsbs/pkg/targets/tdengine/commonpool" +) + +var ( + user string + pass string + host string + port int + runner *query.BenchmarkRunner +) + +func init() { + var config query.BenchmarkRunnerConfig + config.AddToFlagSet(pflag.CommandLine) + + pflag.String("user", "root", "User to connect to TDengine") + pflag.String("pass", "taosdata", "Password for the user connecting to TDengine") + pflag.String("host", "", "TDengine host") + pflag.Int("port", 6030, "TDengine Port") + pflag.Parse() + err := utils.SetupConfigFile() + + if err != nil { + panic(fmt.Errorf("fatal error config file: %s", err)) + } + if err := viper.Unmarshal(&config); err != nil { + panic(fmt.Errorf("unable to decode config: %s", err)) + } + user = viper.GetString("user") + pass = viper.GetString("pass") + host = viper.GetString("host") + port = viper.GetInt("port") + runner = query.NewBenchmarkRunner(config) +} +func main() { + runner.Run(&query.TDenginePool, newProcessor) +} + +type queryExecutorOptions struct { + debug bool + printResponse bool +} + +type processor struct { + db *commonpool.Conn + opts *queryExecutorOptions +} + +func (p *processor) Init(workerNum int) { + async.Init() + db, err := commonpool.GetConnection(user, pass, host, port) + if err != nil { + panic(err) + } + dbName := runner.DatabaseName() + err = async.GlobalAsync.TaosExecWithoutResult(db.TaosConnection, "use "+dbName) + if err != nil { + panic(err) + } + p.db = db + p.opts = &queryExecutorOptions{ + debug: runner.DebugLevel() > 0, + printResponse: runner.DoPrintResponses(), + } +} + +func (p *processor) ProcessQuery(q query.Query, _ bool) ([]*query.Stat, error) { + tq := q.(*query.TDengine) + + start := time.Now() + qry := string(tq.SqlQuery) + if p.opts.debug { + fmt.Println(qry) + } + querys := strings.Split(qry, ";") + if len(querys) > 1 { + var preQuerys []string + for i := 0; i < len(querys); i++ { + if len(querys[i]) > 0 { + preQuerys = append(preQuerys, querys[i]) + } + } + if len(preQuerys) > 1 { + for i := 0; i < len(preQuerys)-1; i++ { + err := async.GlobalAsync.TaosExecWithoutResult(p.db.TaosConnection, preQuerys[i]) + if err != nil { + return nil, err + } + } + } + qry = querys[len(preQuerys)-1] + } + data, err := async.GlobalAsync.TaosExec(p.db.TaosConnection, qry, func(ts int64, precision int) driver.Value { + return ts + }) + if err != nil { + return nil, err + } + if p.opts.printResponse { + fmt.Printf("%#v\n", data) + } + took := float64(time.Since(start).Nanoseconds()) / 1e6 + stat := query.GetStat() + stat.Init(q.HumanLabelName(), took) + + return []*query.Stat{stat}, err +} + +func newProcessor() query.Processor { return &processor{} } diff --git a/docs/tdengine.md b/docs/tdengine.md new file mode 100644 index 000000000..42fbefccb --- /dev/null +++ b/docs/tdengine.md @@ -0,0 +1,143 @@ +# Installation +**`Notice`** + +TDengine uses the Go connector to perform data writing and querying in the +TSBS test. The installation process relies on the C client library. +Therefore, before compiling TSBS, the Taos client must be installed to +provide the necessary library files. For instructions on how to install the +Taos client and server, refer to the [TDengine documentation](https://docs.tdengine.com/get-started/deploy-from-package/). + +# TSBS Supplemental Guide: TDengine + +TDengine is an open-source time-series database with high performance, scalability, and SQL support. +This supplemental guide explains how +the data generated for TSBS is stored, additional flags available when +using the data importer (`tsbs_load_tdenginestmt2` or `tsbs_load_tdengine`), and additional flags +available for the query runner (`tsbs_run_queries_tdengine`). + +**This guide should be read *after* the main README.** + +--- +## Generate Data + +TDengine supports two different write modes for data ingestion: SQL mode and `stmt2` mode. Each mode corresponds to a specific data format generated by `tsbs_generate_data`. It is important to choose the appropriate mode and data format based on your use case. + +### Data Format + +The data format generated by `tsbs_generate_data` depends on the `--format` parameter: + +#### 1. SQL mode: **`--format="TDengine"`**: +Data is serialized in a "pseudo-CSV" format. This format is based on the SQL way of inserting. Each reading consists of a row, where the first item represents the operation type, indicated by 1, 2, or 3. +- **1**: Insert data. The format is: + - `1,common table name,field count,insert data` +- **2**: Create a super table. The format is: + - `2,super table name,common table name,create table SQL` +- **3**: Create a regular table. The format is: + - `3,super table name,common table name,create table SQL` +* Example for the `cpu-only` use case: +```text +2,cpu,host_0,create table cpu (ts timestamp,usage_user bigint,usage_system bigint,usage_idle bigint,usage_nice bigint,usage_iowait bigint,usage_irq bigint,usage_softirq bigint,usage_steal bigint,usage_guest bigint,usage_guest_nice bigint) tags (hostname binary(30),region binary(30),datacenter binary(30),rack binary(30),os binary(30),arch binary(30),team binary(30),service binary(30),service_version binary(30),service_environment binary(30)) +3,cpu,host_0,create table host_0 using cpu (hostname,region,datacenter,rack,os,arch,team,service,service_version,service_environment) tags ('host_0','eu-central-1','eu-central-1a','6','Ubuntu15.10','x86','SF','19','1','test') +1,host_0,10,(1451606400000,58,2,24,61,22,63,6,44,80,38) +``` + +#### 2. `stmt2` mode: **`--format="TDengineStmt2"`**: + +Data is serialized in a binary format optimized for the stmt2 binding method. +This format is more efficient for large-scale data ingestion and is recommended for high-performance scenarios. + +* Example: +The binary format cannot be directly visualized as text but is designed for efficient parsing and insertion using tsbs_load_tdenginestmt2. + +## Load Data +The `tsbs_load_tdengine` and `tsbs_load_tdenginestmt2` tools are used for importing data into TDengine, but they differ in their data insertion methods: + +- **`tsbs_load_tdengine`**: Uses SQL string concatenation to insert data. This method is straightforward and easy to understand but may not achieve the highest performance. +- **`tsbs_load_tdenginestmt2`**: Uses the `stmt2` binding method for data insertion. This approach is more efficient and faster, especially for large-scale data ingestion. + + +## `tsbs_load_tdengine` and `tsbs_load_tdenginestmt2` Additional Flags + +### Connection-Related Flags + +#### `-user` (type: `string`, default: `root`) + +Specifies the username for connecting to TDengine. + +#### `-pass` (type: `string`, default: `taosdata`) + +Specifies the password for the user connecting to TDengine. + +#### `-host` (type: `string`) + +Specifies the hostname of the TDengine server. + +#### `-port` (type: `int`, default: `6030`) + +Specifies the port of the TDengine server. + +### TDengine Database-Related Flags + +TDengine provides several parameters for creating a database. See the [TDengine documentation][conn-str] for more details. + +#### `-buffer` (type: `int`, default: `256`) + +Specifies the size (in MB) of the write buffer for each vnode. Enter a value between 3 and 16384. + +#### `-pages` (type: `int`, default: `256`) + +Specifies the number of pages in the metadata storage engine cache on each vnode. Enter a value greater than or equal to 64. + +#### `-stt_trigger` (type: `int`, default: `1`) + +Specifies the number of file merges triggered by flushed files. The value ranges from 1 to 16. + +#### `-wal_level` (type: `int`, default: `1`) + +Specifies whether fsync is enabled for the Write-Ahead Log (WAL). The default value is 1. + +- **1**: WAL is enabled, but fsync is disabled. +- **2**: Both WAL and fsync are enabled. + +#### `-wal_fsync_period` (type: `int`, default: `3000`) + +Specifies the interval (in milliseconds) at which data is written from the WAL to disk. This parameter is effective only when the WAL parameter is set to 2. Enter a value between 0 and 180000. + +#### `-vgroups` (type: `int`, default: `2`) + +Specifies the initial number of vgroups when a database is created. The value ranges from 0 to 4096. + +#### `-db-name` (type: `string`, default: `benchmark`) + +Specifies the name of the database. + +### Miscellaneous Flags + +#### `-hash-workers` (type: `boolean`, default: `false`) + +Determines whether to consistently hash data across multiple insert workers based on the value of the primary (first) tag. For datasets with a large number of devices, this option helps improve data locality on disk, which can lead to better query performance. For datasets with a small number of devices, this option is typically unnecessary. + +[conn-str]: https://docs.tdengine.com/tdengine-reference/sql-manual/manage-databases/ + +--- + +## `tsbs_run_queries_tdengine` Additional Flags + +### Connection-Related Flags + +#### `-user` (type: `string`, default: `root`) + +Specifies the username for connecting to TDengine. + +#### `-pass` (type: `string`, default: `taosdata`) + +Specifies the password for the user connecting to TDengine. + +#### `-host` (type: `string`) + +Specifies the hostname of the TDengine server. + +#### `-port` (type: `int`, default: `6030`) + +Specifies the port of the TDengine server. + diff --git a/go.mod b/go.mod index 1106116ed..e47a8008c 100644 --- a/go.mod +++ b/go.mod @@ -5,32 +5,56 @@ go 1.14 require ( github.com/HdrHistogram/hdrhistogram-go v1.0.0 github.com/SiriDB/go-siridb-connector v0.0.0-20190110105621-86b34c44c921 - github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d // indirect + github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46 // indirect + github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15 // indirect github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883 - github.com/aws/aws-sdk-go v1.35.13 + github.com/aws/aws-sdk-go v1.38.69 + github.com/bitly/go-hostpool v0.1.0 // indirect github.com/blagojts/viper v1.6.3-0.20200313094124-068f44cf5e69 + github.com/frankban/quicktest v1.11.3 // indirect + github.com/fsnotify/fsnotify v1.5.1 // indirect github.com/globalsign/mgo v0.0.0-20181015135952-eeefdecb41b8 github.com/go-ole/go-ole v1.2.4 // indirect + github.com/go-sql-driver/mysql v1.6.0 // indirect github.com/gocql/gocql v0.0.0-20190810123941-df4b9cc33030 - github.com/golang/protobuf v1.4.2 - github.com/golang/snappy v0.0.1 - github.com/google/flatbuffers v1.11.0 - github.com/google/go-cmp v0.5.2 + github.com/gofrs/uuid v3.3.0+incompatible // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.2 + github.com/golang/snappy v0.0.3 + github.com/google/flatbuffers v2.0.0+incompatible + github.com/google/go-cmp v0.5.6 github.com/jackc/pgx/v4 v4.8.0 github.com/jmoiron/sqlx v1.2.1-0.20190826204134-d7d95172beb5 github.com/kshvakov/clickhouse v1.3.11 github.com/lib/pq v1.3.0 + github.com/magiconair/properties v1.8.5 // indirect + github.com/mattn/go-sqlite3 v1.14.6 // indirect + github.com/mitchellh/mapstructure v1.4.2 // indirect + github.com/pelletier/go-toml v1.9.4 // indirect + github.com/pierrec/lz4 v2.5.2+incompatible // indirect github.com/pkg/errors v0.9.1 - github.com/prometheus/common v0.13.0 - github.com/shirou/gopsutil v3.21.3+incompatible + github.com/prometheus/common v0.26.0 + github.com/shirou/gopsutil v3.21.8+incompatible + github.com/shopspring/decimal v1.2.0 // indirect + github.com/silenceper/pool v1.0.0 + github.com/sirupsen/logrus v1.8.1 // indirect + github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 + github.com/spf13/afero v1.6.0 // indirect + github.com/spf13/cast v1.4.1 // indirect github.com/spf13/cobra v1.0.0 + github.com/spf13/jwalterweatherman v1.1.0 // indirect github.com/spf13/pflag v1.0.5 + github.com/taosdata/driver-go/v3 v3.6.0 github.com/timescale/promscale v0.0.0-20201006153045-6a66a36f5c84 github.com/tklauser/go-sysconf v0.3.5 // indirect github.com/transceptor-technology/go-qpack v0.0.0-20190116123619-49a14b216a45 - github.com/valyala/fasthttp v1.15.1 - go.uber.org/atomic v1.6.0 - golang.org/x/net v0.0.0-20200904194848-62affa334b73 - golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e - gopkg.in/yaml.v2 v2.3.0 + github.com/valyala/fasthttp v1.34.0 + go.uber.org/atomic v1.7.0 + golang.org/x/net v0.7.0 + golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac + google.golang.org/genproto v0.0.0-20210921142501-181ce0d877f6 // indirect + gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect + gopkg.in/ini.v1 v1.63.2 // indirect + gopkg.in/yaml.v2 v2.4.0 + gopkg.in/yaml.v3 v3.0.0 // indirect ) diff --git a/go.sum b/go.sum index 6b3ac40ef..fe4275d6c 100644 --- a/go.sum +++ b/go.sum @@ -46,7 +46,6 @@ github.com/Azure/go-autorest/logger v0.1.0/go.mod h1:oExouG+K6PryycPJfVSxi/koC6L github.com/Azure/go-autorest/logger v0.2.0/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZmbF5NWuPV8+WeEW8= github.com/Azure/go-autorest/tracing v0.5.0/go.mod h1:r/s2XiOKccPW3HrqB+W0TQzfbtp2fGCgRFtBroKn4Dk= github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= -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/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= @@ -74,8 +73,8 @@ github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMx github.com/SiriDB/go-siridb-connector v0.0.0-20190110105621-86b34c44c921 h1:GIWNb0z3t/YKr7xcGNhFgxasaTpnsX91Z0Zt4CeLk+c= github.com/SiriDB/go-siridb-connector v0.0.0-20190110105621-86b34c44c921/go.mod h1:s0x47OhsrJKfg9Iq5orGCVJQjwKklC3jZMFlgLe6Zew= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46 h1:5sXbqlSomvdjlRbWyNqkPsJ3Fg+tQZCbgeX1VGljbQY= +github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= 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/agnivade/levenshtein v1.0.1/go.mod h1:CURSv5d9Uaml+FovSIICkLbAUZ9S4RqaHDIsdSBg7lM= @@ -85,12 +84,14 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafo 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 h1:UQZhZ2O0vMHr2cI+DC1Mbh0TJxzA3RcLoMsFw+aXw7E= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= +github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15 h1:AUNCr9CiJuwrRYS3XieqF+Z9B9gNxo/eANAJCF2eiN4= +github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883 h1:bvNMNQO63//z+xNgfBlViaCIJKLlCJ6/fmUseuG0wVQ= github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883/go.mod h1:rCTlJbsFo29Kk6CurOXKm700vrz8f0KW0JNfpkRJY/8= -github.com/andybalholm/brotli v1.0.0 h1:7UCwP93aiSfvWpapti8g88vVVGp2qqtGyePsSuDafo4= github.com/andybalholm/brotli v1.0.0/go.mod h1:loMXtMfwqflxFJPmdbJO0a3KNoPuLBgiu3qAvBg8x/Y= +github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY= +github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db/go.mod h1:VTxUBvSJ3s3eHAg65PNgrsn5BtqCRPdmyXh6rAfdxN0= github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= @@ -108,15 +109,16 @@ github.com/asaskevich/govalidator v0.0.0-20200108200545-475eaeb16496/go.mod h1:o 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 v1.34.9/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= -github.com/aws/aws-sdk-go v1.35.13 h1:Y49GifH2czbooBMkVpoXwokur1JRBFKVLVCQzO0YsW8= -github.com/aws/aws-sdk-go v1.35.13/go.mod h1:tlPOdRjfxPBpNIwqDj61rmsnA85v9jc0Ps9+muhnW+k= +github.com/aws/aws-sdk-go v1.38.69 h1:V489lmrdkIQSfF6OAGZZ1Cavcm7eczCm2JcGvX+yHRg= +github.com/aws/aws-sdk-go v1.38.69/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= 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/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932 h1:mXoPYz/Ul5HYEDvkta6I8/rnYM5gSdSV2tJ6XbZuEtY= github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932/go.mod h1:NOuUCSz6Q9T7+igc/hlvDOUdtWKryOrtFyIVABv/p7k= +github.com/bitly/go-hostpool v0.1.0 h1:XKmsF6k5el6xHG3WPJ8U0Ku/ye7njX7W81Ng7O2ioR0= +github.com/bitly/go-hostpool v0.1.0/go.mod h1:4gOCgp6+NZnVqlKyZ/iBZFTAJKembaVENUpMkpg42fw= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= @@ -145,6 +147,8 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/apd v1.1.0 h1:3LFP3629v+1aKXU5Q37mxmRxX/pIu1nijXydLShEq5I= github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= @@ -197,6 +201,8 @@ github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4s github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/evanphx/json-patch v0.0.0-20200808040245-162e5629780b/go.mod h1:NAJj0yf/KaRKURN6nyi7A9IZydMivZEm9oQLWNjfKDc= github.com/evanphx/json-patch v4.2.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= @@ -205,9 +211,12 @@ github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= 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/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= +github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/fsnotify/fsnotify v1.5.1 h1:mZcQUHVQUQWoPXXtuf9yuEXKudkV2sx1E06UadKWpgI= +github.com/fsnotify/fsnotify v1.5.1/go.mod h1:T3375wBYaZdLLcVNkcVbzGHY7f1l/uK5T5Ai1i3InKU= 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/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= @@ -224,6 +233,7 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2 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-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= 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= @@ -296,8 +306,9 @@ github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GO github.com/go-redis/redis v6.15.7+incompatible/go.mod h1:NAIEuMOZ/fxfXJIrKDQDz8wamY7mA7PouImQ2Jvg6kA= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-sql-driver/mysql v1.5.0 h1:ozyZYNQW3x3HtqT1jira07DN2PArx2v7/mN66gGcHOs= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= +github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= +github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-toolsmith/astcast v1.0.0/go.mod h1:mt2OdQTeAQcY4DQgPSArJjHCcOwlX+Wl/kwN+LbLGQ4= github.com/go-toolsmith/astcopy v1.0.0/go.mod h1:vrgyG+5Bxrnz4MZWPF+pI4R8h3qKRjjyvV/DSez4WVQ= @@ -338,14 +349,16 @@ github.com/gobwas/glob v0.2.3/go.mod h1:d3Ez4x06l9bZtSvzIay5+Yzi0fmZzPgnTbPcKjJA github.com/gocql/gocql v0.0.0-20190810123941-df4b9cc33030 h1:mqUk3AueyxYmzrE0nu29YlwjmuaWt2sUwk+CrTEGbmY= github.com/gocql/gocql v0.0.0-20190810123941-df4b9cc33030/go.mod h1:Q7Sru5153KG8D9zwueuQJB3ccJf9/bIwF/x8b3oKgT8= github.com/gofrs/flock v0.8.0/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= -github.com/gofrs/uuid v3.2.0+incompatible h1:y12jRkkFxsd7GpqdSZ+/KCs/fJbqpEXSGd4+jfEaewE= github.com/gofrs/uuid v3.2.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= +github.com/gofrs/uuid v3.3.0+incompatible h1:8K4tyRfvU1CYPgJsveYFQMhpFd/wXNM7iK6rR7UHz84= +github.com/gofrs/uuid v3.3.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= github.com/gogo/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.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -371,12 +384,16 @@ github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrU 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.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= -github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.0-20170215233205-553a64147049/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA= +github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2/go.mod h1:k9Qvh+8juN+UKMCS/3jFtGICgW8O96FVaZsaxdzDkR4= github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a/go.mod h1:ryS0uhF+x9jgbj/N71xsEqODy9BN81/GonCZiOzirOk= github.com/golangci/errcheck v0.0.0-20181223084120-ef45e06d44b6/go.mod h1:DbHgvLiFKX1Sh2T1w8Q/h4NAI8MHIpzCdnBUDTXU3I0= @@ -394,15 +411,19 @@ github.com/golangci/revgrep v0.0.0-20180526074752-d9c87f5ffaf0/go.mod h1:qOQCunE github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4/go.mod h1:Izgrg8RkN3rCIMLGE9CyYmU9pY2Jer6DgANEnZ/L/cQ= 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/flatbuffers v1.11.0 h1:O7CEyB8Cb3/DmtxODGtLHcEvpr81Jm5qLg/hsHnxA2A= github.com/google/flatbuffers v1.11.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= +github.com/google/flatbuffers v2.0.0+incompatible h1:dicJ2oXwypfwUGnB2/TYWYEKiuk9eYQlQO/AnOHl5mI= +github.com/google/flatbuffers v2.0.0+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= 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/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.6 h1:BKbKCqvP6I+rmFHt06ZmyQtvB8xAkWdhFyr0ZUNZcxQ= +github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.1.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= @@ -416,6 +437,9 @@ github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hf 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/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= +github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gnostic v0.0.0-20170729233727-0c5108395e2d/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY= @@ -424,7 +448,6 @@ github.com/googleapis/gnostic v0.4.0/go.mod h1:on+2t9HRStVgn95RSsFWFz+6Q0Snyqv1a github.com/gookit/color v1.2.5/go.mod h1:AhIE+pS6D4Ql0SQWbBeXPHw7gY0/sjHoA4s/n1KB7xg= github.com/gophercloud/gophercloud v0.1.0/go.mod h1:vxM41WHh5uqHVBMZHzuwNOHh8XEoIEcSTewFxm1c5g8= github.com/gophercloud/gophercloud v0.12.0/go.mod h1:gmC5oQqMDOMO1t1gq5DquX/yAU808e/4mzjjDA76+Ss= -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= @@ -432,6 +455,8 @@ github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2z github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/gorilla/websocket v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= +github.com/gorilla/websocket v1.5.0/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gostaticanalysis/analysisutil v0.0.0-20190318220348-4088753ea4d3/go.mod h1:eEOZF4jCKGi+aprrirO9e7WKB3beBRtWgqGunKl6pKE= github.com/gostaticanalysis/analysisutil v0.0.3/go.mod h1:eEOZF4jCKGi+aprrirO9e7WKB3beBRtWgqGunKl6pKE= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= @@ -441,8 +466,9 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgf 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/grpc-ecosystem/grpc-gateway v1.14.6/go.mod h1:zdiPV4Yse/1gnckTHtghG4GkDEdKCRJduHpTxT3/jcw= -github.com/grpc-ecosystem/grpc-gateway v1.14.8 h1:hXClj+iFpmLM8i3lkO6i4Psli4P2qObQuQReiII26U8= github.com/grpc-ecosystem/grpc-gateway v1.14.8/go.mod h1:NZE8t6vs6TnwLL/ITkaK8W3ecMLGAbh2jXTclvpiwYo= +github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed h1:5upAirOpQc1Q53c0bnx2ufif5kANL7bfZWcc6VJWJd8= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed/go.mod h1:tMWxXQ9wFIaZeTI9F+hmhFiGpFmhOHzyShyFUhRm0H4= github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= @@ -572,10 +598,11 @@ github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/u 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/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jsternberg/zap-logfmt v1.0.0/go.mod h1:uvPs/4X51zdkcm5jXl5SYoN+4RK21K8mysFmDaM/h+o= -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= @@ -585,23 +612,26 @@ github.com/karrick/godirwalk v1.8.0/go.mod h1:H5KPZjojv4lE+QYImBI8xVtrBRgYrIVsaR github.com/karrick/godirwalk v1.10.3/go.mod h1:RoGL9dQei4vP9ilrpETWE8CLOZ1kiN0LhBygSwrAsHA= 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/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.10.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/compress v1.10.10 h1:a/y8CglcM7gLGYmlbP/stPE5sR3hbhFRUjCBfd/0B3I= github.com/klauspost/compress v1.10.10/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/compress v1.15.0 h1:xqfchp4whNFxn5A4XFyyYtitiWI8Hy5EW59jEwcyL6U= +github.com/klauspost/compress v1.15.0/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPRLkC4NPOvfYeR5KNOrY6TD+/sAC3HXPZgDYg= github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/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.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= +github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA= github.com/kr/pty v1.1.8/go.mod h1:O1sed60cT9XZ5uDucP5qwvh+TE3NnUj51EiZO/lmSfw= @@ -623,8 +653,9 @@ github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0U github.com/logrusorgru/aurora v0.0.0-20181002194514-a7b3b318ed4e/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= 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/magiconair/properties v1.8.5 h1:b6kJs+EmPFMYGkow9GiUyCyOvIwYetYJ3fSaWak/Gls= +github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= github.com/mailru/easyjson v0.0.0-20160728113105-d5b7844b561a/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -654,8 +685,9 @@ github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Ky github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= -github.com/mattn/go-sqlite3 v1.11.0 h1:LDdKkqtYlom37fkvqs8rMPFKAMe8+SgjbwZ6ex1/A/Q= github.com/mattn/go-sqlite3 v1.11.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/mattn/go-sqlite3 v1.14.6 h1:dNPt6NO46WmLVt2DLNpwczCmdV5boIZ6g/tlDrlRUbg= +github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE= github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -673,12 +705,16 @@ github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS4 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/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/mitchellh/mapstructure v1.2.2 h1:dxe5oCinTXiTIcfgmZecdCzPmAJKd46KsCWc35r0TV4= github.com/mitchellh/mapstructure v1.2.2/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/mapstructure v1.4.2 h1:6h7AQ0yhTcIsmFmnAwQls75jp2Gzs4iB8W7pjMO+rqo= +github.com/mitchellh/mapstructure v1.4.2/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= 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-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/morikuni/aec v0.0.0-20170113033406-39771216ff4c/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= @@ -697,7 +733,6 @@ github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxzi 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/nbutton23/zxcvbn-go v0.0.0-20180912185939-ae427f1e4c1d/go.mod h1:o96djdrsSGy3AWPyBgZMAGfxZNfgntdJG+11KU4QvbU= -github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nishanths/exhaustive v0.0.0-20200811152831-6cf413ae40e0/go.mod h1:wBEpHwM2OdmeNpdCvRPUlkEbBuaFmcK4Wv8Q7FuGW3c= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= @@ -741,21 +776,24 @@ github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144T github.com/paulbellamy/ratecounter v0.2.0/go.mod h1:Hfx1hDpSGoqxkVVpBi/IlYD7kChlfo5C6hzIHwPqfFE= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pelletier/go-toml v1.4.0 h1:u3Z1r+oOXJIkxqw34zVhyPgjBsm6X2wn21NWs/HfSeg= github.com/pelletier/go-toml v1.4.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= +github.com/pelletier/go-toml v1.9.4 h1:tjENF6MfZAg8e4ZmZTeWaWiT2vXtsoO6+iuOjFhECwM= +github.com/pelletier/go-toml v1.9.4/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= github.com/peterbourgon/diskv v2.0.1+incompatible/go.mod h1:uqqh8zWWbv1HBMNONnaR/tNboyR3/BZd58JJSHlUSCU= github.com/peterh/liner v1.0.1-0.20180619022028-8c1271fcf47f/go.mod h1:xIteQHvHuaLYG9IFj6mSxM0fCKrs34IrEQUhOYuGPHc= github.com/phayes/checkstyle v0.0.0-20170904204023-bfd46e6a821d/go.mod h1:3OzsM7FXDQlpCiw2j81fOmAwQLnZnLGXVKUzeKQXIAw= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc= -github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +github.com/pierrec/lz4 v2.5.2+incompatible h1:WCjObylUIOlKy/+7Abdn34TLIkXiA4UWUMhxq9m9ZXI= +github.com/pierrec/lz4 v2.5.2+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 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= 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/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pkg/term v0.0.0-20180730021639-bffc007b7fd5/go.mod h1:eCbImbZ95eXtAUIbLAuAVnBnwf83mjf6QIVH8SHYwqQ= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= @@ -784,8 +822,9 @@ github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+ github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.13.0 h1:vJlpe9wPgDRM1Z+7Wj3zUUjY1nr6/1jNKyl7llliccg= github.com/prometheus/common v0.13.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= +github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= 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= @@ -828,47 +867,50 @@ github.com/sergi/go-diff v1.0.0 h1:Kpca3qRNrduNnOQeazBd0ysaKrUJiIuISHxogkT9RPQ= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shazow/go-diff v0.0.0-20160112020656-b6b7b6733b8c/go.mod h1:/PevMnwAxekIXwN8qQyfc5gl2NlkB3CQlkizAbOkeBs= github.com/shirou/gopsutil v0.0.0-20190901111213-e4ec7b275ada/go.mod h1:WWnYX4lzhCH5h/3YBfyVA3VbLYjlMZZAQcW9ojMexNc= -github.com/shirou/gopsutil v2.18.12+incompatible h1:1eaJvGomDnH74/5cF4CTmTbLHAriGFsTZppLXDX93OM= -github.com/shirou/gopsutil v2.18.12+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v3.21.3+incompatible h1:uenXGGa8ESCQq+dbgtl916dmg6PSAz2cXov0uORQ9v8= -github.com/shirou/gopsutil v3.21.3+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4 h1:udFKJ0aHUL60LboW/A+DfgoHVedieIzIXE8uylPue0U= +github.com/shirou/gopsutil v3.21.8+incompatible h1:sh0foI8tMRlCidUJR+KzqWYWxrkuuPIGiO6Vp+KXdCU= +github.com/shirou/gopsutil v3.21.8+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= -github.com/shopspring/decimal v0.0.0-20200227202807-02e2044944cc h1:jUIKcSPO9MoMJBbEoyE/RJoE8vz7Mb8AjvifMMwSyvY= github.com/shopspring/decimal v0.0.0-20200227202807-02e2044944cc/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= +github.com/shopspring/decimal v1.2.0 h1:abSATXmQEYyShuxI4/vyW3tV1MrKAJzCZ/0zLUXYbsQ= +github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041/go.mod h1:N5mDOmsrJOB+vfqUK+7DmDyjhSLIIBnXo9lvZJj3MWQ= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/shurcooL/vfsgen v0.0.0-20200627165143-92b8a710ab6c/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= +github.com/silenceper/pool v1.0.0 h1:JTCaA+U6hJAA0P8nCx+JfsRCHMwLTfatsm5QXelffmU= +github.com/silenceper/pool v1.0.0/go.mod h1:3DN13bqAbq86Lmzf6iUXWEPIWFPOSYVfaoceFvilKKI= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= +github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= +github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= 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/sonatard/noctx v0.0.1/go.mod h1:9D2D/EoULe8Yy2joDHJj7bv3sZoq9AaSb8B4lqBjiZI= github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= github.com/sourcegraph/go-diff v0.6.0/go.mod h1:iBszgVvyxdc8SFZ7gm69go2KDdt3ag071iBaWPF6cjs= +github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/afero v1.2.2 h1:5jhuqJyZCZf2JRofRvN/nIFgIWNzPa3/Vz8mYylgbWc= github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= -github.com/spf13/cast v1.3.0 h1:oget//CVOEoFewqQxwr0Ej5yjygnqGkvggSE/gB35Q8= +github.com/spf13/afero v1.6.0 h1:xoax2sJ2DT8S8xA2paPFjDCScCNeWsg75VG0DLRreiY= +github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +github.com/spf13/cast v1.4.1 h1:s0hze+J0196ZfEMTs80N7UlFt0BDuQ7Q+JDnHiMWKdA= +github.com/spf13/cast v1.4.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.7/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= -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/jwalterweatherman v1.1.0 h1:ue6voC5bR5F8YxI5S67j9i582FU4Qvo2bmqnqMYADFk= +github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= 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= @@ -888,10 +930,13 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf 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.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -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/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= +github.com/stretchr/testify v1.7.0/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/taosdata/driver-go/v3 v3.6.0 h1:4dRXMl01DhIS5xBXUvtkkB+MjL8g64zN674xKd+ojTE= +github.com/taosdata/driver-go/v3 v3.6.0/go.mod h1:H2vo/At+rOPY1aMzUV9P49SVX7NlXb3LAbKw+MCLrmU= github.com/tdakkota/asciicheck v0.0.0-20200416190851-d7f85be797a2/go.mod h1:yHp0ai0Z9gUljN3o0xMhYJnH/IcvkdTBOX2fmJ93JEM= github.com/testcontainers/testcontainers-go v0.5.1/go.mod h1:Oc/G02bjZiX0p3lzyh6b1GCELP0e4/6Cg3ciU/LnFvU= github.com/tetafro/godot v0.4.8/go.mod h1:/7NLHhv08H1+8DNj0MElpAACw1ajsCuf3TKNQxA5S+0= @@ -902,6 +947,7 @@ github.com/timescale/promscale v0.0.0-20201006153045-6a66a36f5c84/go.mod h1:rkhy github.com/tinylib/msgp v1.0.2/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tklauser/go-sysconf v0.3.5 h1:uu3Xl4nkLzQfXNsWn15rPc/HQCJKObbt1dKJeWp3vU4= github.com/tklauser/go-sysconf v0.3.5/go.mod h1:MkWzOF4RMCshBAMXuhXJs64Rte09mITnppBXY/rYEFI= +github.com/tklauser/numcpus v0.2.2 h1:oyhllyrScuYI6g+h/zUvNXNp1wy7x8qQy3t/piefldA= github.com/tklauser/numcpus v0.2.2/go.mod h1:x3qojaO3uyYt0i56EW/VUYs7uBvdl2fkfZFu0T9wgjM= 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= @@ -921,10 +967,12 @@ github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtX github.com/uudashr/gocognit v1.0.1/go.mod h1:j44Ayx2KW4+oB6SWMv8KsmHzZrOInQav7D3cQMJ5JUM= 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.15.1 h1:eRb5jzWhbCn/cGu3gNJMcOfPUfXgXCcQIOHjh9ajAS8= github.com/valyala/fasthttp v1.15.1/go.mod h1:YOKImeEosDdBPnxc0gy7INqi3m1zK6A+xl6TwOBhHCA= +github.com/valyala/fasthttp v1.34.0 h1:d3AAQJ2DRcxJYHm7OXNXtXt2as1vMDfxeIcFvhmGGm4= +github.com/valyala/fasthttp v1.34.0/go.mod h1:epZA5N+7pY6ZaEKRmstzOuYJx9HI8DI1oaCGZpdH4h0= github.com/valyala/quicktemplate v1.6.2/go.mod h1:mtEJpQtUiBV0SHhMX6RtiJtqxncgrfmjcUy5T68X8TM= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= +github.com/valyala/tcplisten v1.0.0/go.mod h1:T0xQ8SeCZGxckz9qRXTfG43PvQ/mcWh7FwZEA7Ioqkc= github.com/vektah/gqlparser v1.1.2/go.mod h1:1ycwN7Ij5njmMkPPAOaRFY4rET2Enx7IkVv3vaXspKw= github.com/willf/bitset v1.1.3/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= @@ -937,6 +985,7 @@ github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= @@ -952,11 +1001,13 @@ 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.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= 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/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= @@ -986,8 +1037,10 @@ golang.org/x/crypto v0.0.0-20191202143827-86a70503ff7e/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200220183623-bac4c82f6975/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rBCcS0QyQY66Mpf/7BZbInM= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20220214200702-86341886e292 h1:f+lwQ+GtmgoY+A2YaQxlSOnDjXcQ7ZRLWOHbC6HtRqE= +golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1013,7 +1066,6 @@ golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= 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= @@ -1022,8 +1074,8 @@ 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/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= 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= @@ -1063,8 +1115,16 @@ golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/ golang.org/x/net v0.0.0-20200602114024-627f9648deb9/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200904194848-62affa334b73 h1:MXfv8rhZWmFeqX3GNZRsd6vOLoaCHjYEX3qkRo3YBUA= golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= +golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= 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= @@ -1079,6 +1139,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/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= @@ -1136,23 +1198,42 @@ golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7w 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-20200821140526-fda516888d29/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200908134130-d2e65c121b96 h1:gJciq3lOg0eS9fSZJcoHfv7q1BfC6cJfnmSSKL1yu3Q= golang.org/x/sys v0.0.0-20200908134130-d2e65c121b96/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210316164454-77fc1eacc6aa h1:ZYxPR6aca/uhfRJyaOAtflSHjJYiktO7QnJC5ut7iY4= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210316164454-77fc1eacc6aa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220227234510-4e6760a101f9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/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/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= 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/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s= golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac h1:7zkz7BUtwNFFqcowJ+RIgu2MaV/MapERkDIy+mwPyjs= +golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180810170437-e96c4e24768d/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1222,14 +1303,16 @@ golang.org/x/tools v0.0.0-20200414032229-332987a829c3/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200422022333-3d57cf2e726e/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200513201620-d5fe73897c97/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200519015757-0d0afa43d58a/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200625211823-6506e20df31f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200626171337-aa94e735be7f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200701041122-1837592efa10/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200724022722-7017fd6b1305/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200812195022-5ae4c3c160a0/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200822203824-307de81be3f4/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200908211811-12e1bf57a112 h1:DmrRJy1qn9VDMf4+GSpRlwfZ51muIF7r96MFBFP4bPM= golang.org/x/tools v0.0.0-20200908211811-12e1bf57a112/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190513163551-3ee3066db522/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1286,8 +1369,9 @@ google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfG google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20200815001618-f69a88009b70/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d h1:92D1fum1bJLKSdr11OJ+54YeCMCGYIygTA7R/YZxH5M= google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210921142501-181ce0d877f6 h1:2ncG/LajxmrclaZH+ppVi02rQxz4eXYJzGHdFN4Y9UA= +google.golang.org/genproto v0.0.0-20210921142501-181ce0d877f6/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= 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= @@ -1304,8 +1388,11 @@ google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.32.0 h1:zWTV+LMdc3kaiJMSTOFz2UgSBgx8RNQoTGiZu3fR9S0= google.golang.org/grpc v1.32.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.40.0 h1:AGJ0Ih4mHjSeibYkFGh1dD9KJ/eOtZ93I6hoHhukQ5Q= +google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= 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= @@ -1315,15 +1402,19 @@ google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2 google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= -google.golang.org/protobuf v1.25.0 h1:Ejskq+SyPohKW+1uil0JJMtmHCgJPJ/qWTxr8qp+R4c= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= +google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= 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/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f h1:BLraFXnmrev5lT+xlilqcH8XK9/i0At2xKjWk4p6zsU= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= @@ -1332,8 +1423,9 @@ gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= gopkg.in/inconshreveable/log15.v2 v2.0.0-20180818164646-67afb5ed74ec/go.mod h1:aPpfJ7XW+gOuirDoZ8gHhLh3kZ1B08FtV2bbmy7Jv3s= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= -gopkg.in/ini.v1 v1.51.0 h1:AQvPpx3LzTDM0AjnIRlVFwFFGC+npRopjZxLJj6gdno= gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/ini.v1 v1.63.2 h1:tGK/CyBg7SMzb60vP1M03vNZ3VDu3wGQJwn7Sxi9r3c= +gopkg.in/ini.v1 v1.63.2/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= @@ -1345,11 +1437,13 @@ 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.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/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.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0 h1:hjy8E9ON/egN1tAYqKb61G10WtihqetD4sz2H+8nIeA= +gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools v0.0.0-20181223230014-1083505acf35/go.mod h1:R//lfYlUuTOTfblYI3lGoAAAebUdzjvbmQsuB7Ykd90= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/internal/inputs/generator_data.go b/internal/inputs/generator_data.go index ee639659c..c42dcb98d 100644 --- a/internal/inputs/generator_data.go +++ b/internal/inputs/generator_data.go @@ -136,10 +136,17 @@ func (g *DataGenerator) getSerializer(sim common.Simulator, target targets.Imple case constants.FormatTimescaleDB: g.writeHeader(sim.Headers()) } - return target.Serializer(), nil + serializer := target.Serializer() + if cs, ok := serializer.(serialize.ConfigurableSerializer); ok { + err := cs.Config(g.config, g.bufOut) + if err != nil { + return nil, err + } + } + return serializer, nil } -//TODO should be implemented in targets package +// TODO should be implemented in targets package func (g *DataGenerator) writeHeader(headers *common.GeneratedDataHeaders) { g.bufOut.WriteString("tags") diff --git a/pkg/data/serialize/point_serializer.go b/pkg/data/serialize/point_serializer.go index e632de797..5f5c5ae24 100644 --- a/pkg/data/serialize/point_serializer.go +++ b/pkg/data/serialize/point_serializer.go @@ -1,11 +1,18 @@ package serialize import ( - "github.com/timescale/tsbs/pkg/data" "io" + + "github.com/timescale/tsbs/pkg/data" + "github.com/timescale/tsbs/pkg/data/usecases/common" ) // PointSerializer serializes a Point for writing type PointSerializer interface { Serialize(p *data.Point, w io.Writer) error } + +type ConfigurableSerializer interface { + PointSerializer + Config(*common.DataGeneratorConfig, io.Writer) error +} diff --git a/pkg/query/factories/init_factories.go b/pkg/query/factories/init_factories.go index ff3faf47d..4deae56f3 100644 --- a/pkg/query/factories/init_factories.go +++ b/pkg/query/factories/init_factories.go @@ -9,6 +9,7 @@ import ( "github.com/timescale/tsbs/cmd/tsbs_generate_queries/databases/mongo" "github.com/timescale/tsbs/cmd/tsbs_generate_queries/databases/questdb" "github.com/timescale/tsbs/cmd/tsbs_generate_queries/databases/siridb" + "github.com/timescale/tsbs/cmd/tsbs_generate_queries/databases/tdengine" "github.com/timescale/tsbs/cmd/tsbs_generate_queries/databases/timescaledb" "github.com/timescale/tsbs/cmd/tsbs_generate_queries/databases/timestream" "github.com/timescale/tsbs/cmd/tsbs_generate_queries/databases/victoriametrics" @@ -39,5 +40,6 @@ func InitQueryFactories(config *config.QueryGeneratorConfig) map[string]interfac DBName: config.DbName, } factories[constants.FormatQuestDB] = &questdb.BaseGenerator{} + factories[constants.FormatTDengine] = &tdengine.BaseGenerator{} return factories } diff --git a/pkg/query/stats.go b/pkg/query/stats.go index 007f41a45..d71914df9 100644 --- a/pkg/query/stats.go +++ b/pkg/query/stats.go @@ -65,6 +65,9 @@ type statGroup struct { latencyHDRHistogram *hdrhistogram.Histogram sum float64 count int64 + intervalHistogram *hdrhistogram.Histogram + intervalSum float64 + intervalCount int64 } // newStatGroup returns a new StatGroup with an initial size @@ -77,9 +80,14 @@ func newStatGroup(size uint64) *statGroup { // - 10 millisecond (or better) from 10 millisecond up to 10 seconds, // - 1 second (or better) from 10 second up to 3600 seconds, lH := hdrhistogram.New(1, 3600000000, 4) + iH := hdrhistogram.New(1, 3600000000, 4) return &statGroup{ - count: 0, latencyHDRHistogram: lH, + sum: 0, + count: 0, + intervalHistogram: iH, + intervalSum: 0, + intervalCount: 0, } } @@ -88,11 +96,14 @@ func (s *statGroup) push(n float64) { s.latencyHDRHistogram.RecordValue(int64(n * hdrScaleFactor)) s.sum += n s.count++ + s.intervalHistogram.RecordValue(int64(n * hdrScaleFactor)) + s.intervalSum += n + s.intervalCount++ } // string makes a simple description of a statGroup. func (s *statGroup) string() string { - return fmt.Sprintf("min: %8.2fms, med: %8.2fms, mean: %8.2fms, max: %7.2fms, stddev: %8.2fms, sum: %5.1fsec, count: %d", + vl := fmt.Sprintf("min: %8.2fms, med: %8.2fms, mean: %8.2fms, max: %7.2fms, stddev: %8.2fms, sum: %5.1fsec, count: %d", s.Min(), s.Median(), s.Mean(), @@ -100,6 +111,21 @@ func (s *statGroup) string() string { s.StdDev(), s.sum/hdrScaleFactor, s.count) + vi := "" + if s.intervalCount != 0 { + vi = fmt.Sprintf("\nmin: %8.2fms, med: %8.2fms, mean: %8.2fms, max: %7.2fms, stddev: %8.2fms, sum: %5.1fsec, count: %d", + s.MinInterval(), + s.MedianInterval(), + s.MeanInterval(), + s.MaxInterval(), + s.StdDevInterval(), + s.intervalSum/hdrScaleFactor, + s.intervalCount) + } + s.intervalSum = 0 + s.intervalCount = 0 + s.intervalHistogram.Reset() + return vl + vi } func (s *statGroup) write(w io.Writer) error { @@ -132,6 +158,26 @@ func (s *statGroup) StdDev() float64 { return float64(s.latencyHDRHistogram.StdDev()) / hdrScaleFactor } +func (s *statGroup) MedianInterval() float64 { + return float64(s.intervalHistogram.ValueAtQuantile(50.0)) / hdrScaleFactor +} + +func (s *statGroup) MeanInterval() float64 { + return float64(s.intervalHistogram.Mean()) / hdrScaleFactor +} + +func (s *statGroup) MaxInterval() float64 { + return float64(s.intervalHistogram.Max()) / hdrScaleFactor +} + +func (s *statGroup) MinInterval() float64 { + return float64(s.intervalHistogram.Min()) / hdrScaleFactor +} + +func (s *statGroup) StdDevInterval() float64 { + return float64(s.intervalHistogram.StdDev()) / hdrScaleFactor +} + // writeStatGroupMap writes a map of StatGroups in an ordered fashion by // key that they are stored by func writeStatGroupMap(w io.Writer, statGroups map[string]*statGroup) error { diff --git a/pkg/query/tdengine.go b/pkg/query/tdengine.go new file mode 100644 index 000000000..a8f842c53 --- /dev/null +++ b/pkg/query/tdengine.go @@ -0,0 +1,59 @@ +package query + +import ( + "fmt" + "sync" +) + +type TDengine struct { + id uint64 + HumanLabel []byte + HumanDescription []byte + Hypertable []byte + SqlQuery []byte +} + +var TDenginePool = sync.Pool{ + New: func() interface{} { + return &TDengine{ + HumanLabel: make([]byte, 0, 1024), + HumanDescription: make([]byte, 0, 1024), + Hypertable: make([]byte, 0, 1024), + SqlQuery: make([]byte, 0, 1024), + } + }, +} + +func NewTDengine() *TDengine { + return TDenginePool.Get().(*TDengine) +} + +func (q *TDengine) Release() { + q.HumanLabel = q.HumanLabel[:0] + q.HumanDescription = q.HumanDescription[:0] + q.id = 0 + + q.Hypertable = q.Hypertable[:0] + q.SqlQuery = q.SqlQuery[:0] + TDenginePool.Put(q) +} + +func (q *TDengine) HumanLabelName() []byte { + return q.HumanLabel +} + +func (q *TDengine) HumanDescriptionName() []byte { + return q.HumanDescription +} + +func (q *TDengine) GetID() uint64 { + return q.id +} + +func (q *TDengine) SetID(n uint64) { + q.id = n +} + +func (q *TDengine) String() string { + return fmt.Sprintf("HumanLabel: %s, HumanDescription: %s, Hypertable: %s, Query: %s", q.HumanLabel, q.HumanDescription, q.Hypertable, q.SqlQuery) +} diff --git a/pkg/targets/constants/constants.go b/pkg/targets/constants/constants.go index 516093d09..e628a1000 100644 --- a/pkg/targets/constants/constants.go +++ b/pkg/targets/constants/constants.go @@ -14,6 +14,8 @@ const ( FormatVictoriaMetrics = "victoriametrics" FormatTimestream = "timestream" FormatQuestDB = "questdb" + FormatTDengine = "TDengine" + FormatTDengineStmt2 = "TDengineStmt2" ) func SupportedFormats() []string { @@ -30,5 +32,7 @@ func SupportedFormats() []string { FormatVictoriaMetrics, FormatTimestream, FormatQuestDB, + FormatTDengine, + FormatTDengineStmt2, } } diff --git a/pkg/targets/initializers/target_initializers.go b/pkg/targets/initializers/target_initializers.go index a7a31677e..f42b24600 100644 --- a/pkg/targets/initializers/target_initializers.go +++ b/pkg/targets/initializers/target_initializers.go @@ -2,6 +2,8 @@ package initializers import ( "fmt" + "strings" + "github.com/timescale/tsbs/pkg/targets" "github.com/timescale/tsbs/pkg/targets/akumuli" "github.com/timescale/tsbs/pkg/targets/cassandra" @@ -13,10 +15,11 @@ import ( "github.com/timescale/tsbs/pkg/targets/prometheus" "github.com/timescale/tsbs/pkg/targets/questdb" "github.com/timescale/tsbs/pkg/targets/siridb" + "github.com/timescale/tsbs/pkg/targets/tdengine" + "github.com/timescale/tsbs/pkg/targets/tdenginestmt2" "github.com/timescale/tsbs/pkg/targets/timescaledb" "github.com/timescale/tsbs/pkg/targets/timestream" "github.com/timescale/tsbs/pkg/targets/victoriametrics" - "strings" ) func GetTarget(format string) targets.ImplementedTarget { @@ -45,6 +48,10 @@ func GetTarget(format string) targets.ImplementedTarget { return timestream.NewTarget() case constants.FormatQuestDB: return questdb.NewTarget() + case constants.FormatTDengine: + return tdengine.NewTarget() + case constants.FormatTDengineStmt2: + return tdenginestmt2.NewTarget() } supportedFormatsStr := strings.Join(constants.SupportedFormats(), ",") diff --git a/pkg/targets/targets.go b/pkg/targets/targets.go index 064bc5ab1..a727772cd 100644 --- a/pkg/targets/targets.go +++ b/pkg/targets/targets.go @@ -71,6 +71,11 @@ type Benchmark interface { GetDBCreator() DBCreator } +type ConfigurableBenchmark interface { + Benchmark + SetConfig(batchSize uint, workers uint) +} + type DataSource interface { NextItem() data.LoadedPoint Headers() *common.GeneratedDataHeaders diff --git a/pkg/targets/tdengine/async/handlerpool.go b/pkg/targets/tdengine/async/handlerpool.go new file mode 100644 index 000000000..1437a367e --- /dev/null +++ b/pkg/targets/tdengine/async/handlerpool.go @@ -0,0 +1,120 @@ +package async + +import ( + "container/list" + "sync" + "unsafe" + + "github.com/taosdata/driver-go/v3/wrapper/cgo" +) + +type Result struct { + Res unsafe.Pointer + N int +} +type Stmt2ExecResult struct { + Res unsafe.Pointer + Affected int + Code int +} +type Caller struct { + QueryResult chan *Result + FetchResult chan *Result + ExecResult chan *Stmt2ExecResult +} + +func NewCaller() *Caller { + return &Caller{ + QueryResult: make(chan *Result, 1), + FetchResult: make(chan *Result, 1), + ExecResult: make(chan *Stmt2ExecResult, 1), + } +} + +func (c *Caller) QueryCall(res unsafe.Pointer, code int) { + c.QueryResult <- &Result{ + Res: res, + N: code, + } +} + +func (c *Caller) FetchCall(res unsafe.Pointer, numOfRows int) { + c.FetchResult <- &Result{ + Res: res, + N: numOfRows, + } +} + +func (c *Caller) ExecCall(res unsafe.Pointer, affected int, code int) { + c.ExecResult <- &Stmt2ExecResult{ + Res: res, + Affected: affected, + Code: code, + } +} + +type poolReq struct { + idleHandler *Handler +} + +type HandlerPool struct { + mu sync.RWMutex + count int + handlers chan *Handler + reqList *list.List +} + +type Handler struct { + Handler cgo.Handle + Caller *Caller +} + +func NewHandlerPool(count int) *HandlerPool { + c := &HandlerPool{ + count: count, + handlers: make(chan *Handler, count), + reqList: list.New(), + } + for i := 0; i < count; i++ { + caller := NewCaller() + c.handlers <- &Handler{ + Handler: cgo.NewHandle(caller), + Caller: caller, + } + } + return c +} + +func (c *HandlerPool) Get() *Handler { + for { + select { + case wrapConn := <-c.handlers: + return wrapConn + default: + c.mu.Lock() + req := make(chan poolReq, 1) + c.reqList.PushBack(req) + c.mu.Unlock() + ret := <-req + return ret.idleHandler + } + } +} + +func (c *HandlerPool) Put(handler *Handler) { + c.mu.Lock() + e := c.reqList.Front() + if e != nil { + req := e.Value.(chan poolReq) + c.reqList.Remove(e) + req <- poolReq{ + idleHandler: handler, + } + c.mu.Unlock() + return + } else { + c.handlers <- handler + c.mu.Unlock() + return + } +} diff --git a/pkg/targets/tdengine/async/row.go b/pkg/targets/tdengine/async/row.go new file mode 100644 index 000000000..bd3e39013 --- /dev/null +++ b/pkg/targets/tdengine/async/row.go @@ -0,0 +1,133 @@ +package async + +import ( + "database/sql/driver" + "errors" + "sync" + "unsafe" + + "github.com/taosdata/driver-go/v3/common/parser" + tErrors "github.com/taosdata/driver-go/v3/errors" + "github.com/taosdata/driver-go/v3/wrapper" + "github.com/timescale/tsbs/pkg/targets/tdengine/thread" +) + +var FetchRowError = errors.New("fetch row error") +var GlobalAsync *Async + +type Async struct { + HandlerPool *HandlerPool +} + +func NewAsync(handlerPool *HandlerPool) *Async { + return &Async{HandlerPool: handlerPool} +} + +func (a *Async) TaosExec(taosConnect unsafe.Pointer, sql string, timeFormat parser.FormatTimeFunc) (*ExecResult, error) { + handler := a.HandlerPool.Get() + defer a.HandlerPool.Put(handler) + result, err := a.TaosQuery(taosConnect, sql, handler) + defer func() { + if result != nil && result.Res != nil { + thread.Lock() + wrapper.TaosFreeResult(result.Res) + thread.Unlock() + } + }() + if err != nil { + return nil, err + } + res := result.Res + code := wrapper.TaosError(res) + if code != int(tErrors.SUCCESS) { + errStr := wrapper.TaosErrorStr(res) + return nil, tErrors.NewError(code, errStr) + } + isUpdate := wrapper.TaosIsUpdateQuery(res) + execResult := &ExecResult{} + if isUpdate { + affectRows := wrapper.TaosAffectedRows(res) + execResult.AffectedRows = affectRows + return execResult, nil + } + fieldsCount := wrapper.TaosNumFields(res) + execResult.FieldCount = fieldsCount + var rowsHeader *wrapper.RowsHeader + rowsHeader, err = wrapper.ReadColumn(res, fieldsCount) + if err != nil { + return nil, err + } + execResult.Header = rowsHeader + precision := wrapper.TaosResultPrecision(res) + for { + result, err = a.TaosFetchRawBlockA(res, handler) + if err != nil { + return nil, err + } + if result.N == 0 { + return execResult, nil + } else if result.N < 0 { + errStr := wrapper.TaosErrorStr(result.Res) + return nil, tErrors.NewError(result.N, errStr) + } else { + res = result.Res + block := wrapper.TaosGetRawBlock(res) + values := parser.ReadBlockWithTimeFormat(block, result.N, rowsHeader.ColTypes, precision, timeFormat) + execResult.Data = append(execResult.Data, values...) + } + } +} + +func (a *Async) TaosQuery(taosConnect unsafe.Pointer, sql string, handler *Handler) (*Result, error) { + thread.Lock() + wrapper.TaosQueryA(taosConnect, sql, handler.Handler) + thread.Unlock() + r := <-handler.Caller.QueryResult + return r, nil +} + +func (a *Async) TaosFetchRawBlockA(res unsafe.Pointer, handler *Handler) (*Result, error) { + thread.Lock() + wrapper.TaosFetchRawBlockA(res, handler.Handler) + thread.Unlock() + r := <-handler.Caller.FetchResult + return r, nil +} + +type ExecResult struct { + AffectedRows int + FieldCount int + Header *wrapper.RowsHeader + Data [][]driver.Value +} + +func (a *Async) TaosExecWithoutResult(taosConnect unsafe.Pointer, sql string) error { + handler := a.HandlerPool.Get() + defer a.HandlerPool.Put(handler) + result, err := a.TaosQuery(taosConnect, sql, handler) + defer func() { + if result != nil && result.Res != nil { + thread.Lock() + wrapper.TaosFreeResult(result.Res) + thread.Unlock() + } + }() + if err != nil { + return err + } + res := result.Res + code := wrapper.TaosError(res) + if code != int(tErrors.SUCCESS) { + errStr := wrapper.TaosErrorStr(res) + return tErrors.NewError(code, errStr) + } + return nil +} + +var once sync.Once + +func Init() { + once.Do(func() { + GlobalAsync = NewAsync(NewHandlerPool(10000)) + }) +} diff --git a/pkg/targets/tdengine/benchmark.go b/pkg/targets/tdengine/benchmark.go new file mode 100644 index 000000000..e25ecd12e --- /dev/null +++ b/pkg/targets/tdengine/benchmark.go @@ -0,0 +1,66 @@ +package tdengine + +import ( + "bytes" + "math" + + "github.com/timescale/tsbs/pkg/data/source" + "github.com/timescale/tsbs/pkg/targets" +) + +func NewBenchmark(dbName string, opts *LoadingOptions, dataSourceConfig *source.DataSourceConfig) (targets.Benchmark, error) { + var ds targets.DataSource + if dataSourceConfig.Type == source.FileDataSourceType { + ds = newFileDataSource(dataSourceConfig.File.Location) + } else { + panic("not implement") + } + + return &benchmark{ + opts: opts, + dataSource: ds, + dbName: dbName, + factory: NewBatchFactory(), + }, nil +} + +type benchmark struct { + opts *LoadingOptions + dataSource targets.DataSource + dbName string + batchSize uint + factory *BatchFactory +} + +func (b *benchmark) GetDataSource() targets.DataSource { + return b.dataSource +} + +func (b *benchmark) GetBatchFactory() targets.BatchFactory { + return b.factory +} + +func (b *benchmark) GetPointIndexer(maxPartitions uint) targets.PointIndexer { + if maxPartitions > 1 { + interval := uint32(math.MaxUint32 / maxPartitions) + hashEndGroups := make([]uint32, maxPartitions) + for i := 0; i < int(maxPartitions); i++ { + if i == int(maxPartitions)-1 { + hashEndGroups[i] = math.MaxUint32 + } else { + hashEndGroups[i] = interval*uint32(i+1) - 1 + } + } + prefix := []byte("1." + b.dbName + ".") + return &indexer{buffer: &bytes.Buffer{}, prefix: prefix, hashEndGroups: hashEndGroups, partitions: int(maxPartitions), tmp: map[string]uint{}} + } + return &targets.ConstantIndexer{} +} + +func (b *benchmark) GetProcessor() targets.Processor { + return newProcessor(b.opts, b.dbName) +} + +func (b *benchmark) GetDBCreator() targets.DBCreator { + return &DbCreator{Opts: b.opts} +} diff --git a/pkg/targets/tdengine/commonpool/pool.go b/pkg/targets/tdengine/commonpool/pool.go new file mode 100644 index 000000000..9c87ad270 --- /dev/null +++ b/pkg/targets/tdengine/commonpool/pool.go @@ -0,0 +1,133 @@ +package commonpool + +import ( + "sync" + "unsafe" + + "github.com/silenceper/pool" + "github.com/taosdata/driver-go/v3/wrapper" + "github.com/timescale/tsbs/pkg/targets/tdengine/thread" +) + +type ConnectorPool struct { + host string + user string + password string + port int + pool pool.Pool +} + +func NewConnectorPool(user, password, host string, port int) (*ConnectorPool, error) { + a := &ConnectorPool{user: user, password: password, host: host, port: port} + poolConfig := &pool.Config{ + InitialCap: 1, + MaxCap: 10000, + MaxIdle: 10000, + Factory: a.factory, + Close: a.close, + IdleTimeout: -1, + } + p, err := pool.NewChannelPool(poolConfig) + if err != nil { + return nil, err + } + a.pool = p + return a, nil +} + +func (a *ConnectorPool) factory() (interface{}, error) { + thread.Lock() + defer thread.Unlock() + return wrapper.TaosConnect(a.host, a.user, a.password, "", a.port) +} + +func (a *ConnectorPool) close(v interface{}) error { + if v != nil { + thread.Lock() + defer thread.Unlock() + wrapper.TaosClose(v.(unsafe.Pointer)) + } + return nil +} + +func (a *ConnectorPool) Get() (unsafe.Pointer, error) { + v, err := a.pool.Get() + if err != nil { + return nil, err + } + return v.(unsafe.Pointer), nil +} + +func (a *ConnectorPool) Put(c unsafe.Pointer) error { + wrapper.TaosResetCurrentDB(c) + return a.pool.Put(c) +} + +func (a *ConnectorPool) Close(c unsafe.Pointer) error { + return a.pool.Close(c) +} + +func (a *ConnectorPool) Release() { + a.pool.Release() +} + +func (a *ConnectorPool) verifyPassword(password string) bool { + return password == a.password +} + +var connectionMap = sync.Map{} + +type Conn struct { + TaosConnection unsafe.Pointer + pool *ConnectorPool +} + +func (c *Conn) Put() error { + return c.pool.Put(c.TaosConnection) +} + +func GetConnection(user, password, host string, port int) (*Conn, error) { + p, exist := connectionMap.Load(user) + if exist { + connectionPool := p.(*ConnectorPool) + if !connectionPool.verifyPassword(password) { + newPool, err := NewConnectorPool(user, password, host, port) + if err != nil { + return nil, err + } + connectionPool.Release() + connectionMap.Store(user, newPool) + c, err := newPool.Get() + if err != nil { + return nil, err + } + return &Conn{ + TaosConnection: c, + pool: newPool, + }, nil + } else { + c, err := connectionPool.Get() + if err != nil { + return nil, err + } + return &Conn{ + TaosConnection: c, + pool: connectionPool, + }, nil + } + } else { + newPool, err := NewConnectorPool(user, password, host, port) + if err != nil { + return nil, err + } + connectionMap.Store(user, newPool) + c, err := newPool.Get() + if err != nil { + return nil, err + } + return &Conn{ + TaosConnection: c, + pool: newPool, + }, nil + } +} diff --git a/pkg/targets/tdengine/creator.go b/pkg/targets/tdengine/creator.go new file mode 100644 index 000000000..7439680ab --- /dev/null +++ b/pkg/targets/tdengine/creator.go @@ -0,0 +1,68 @@ +package tdengine + +import ( + "fmt" + "log" + "strconv" + + "github.com/timescale/tsbs/pkg/targets/tdengine/async" + "github.com/timescale/tsbs/pkg/targets/tdengine/commonpool" +) + +var fatal = log.Fatalf + +type DbCreator struct { + Opts *LoadingOptions + Db *commonpool.Conn +} + +func (d *DbCreator) Init() { + async.Init() + db, err := commonpool.GetConnection(d.Opts.User, d.Opts.Pass, d.Opts.Host, d.Opts.Port) + if err != nil { + panic("TDengine can not get connection") + } + d.Db = db +} + +func (d *DbCreator) DBExists(dbName string) bool { + err := async.GlobalAsync.TaosExecWithoutResult(d.Db.TaosConnection, "use "+dbName) + return err == nil +} + +func (d *DbCreator) CreateDB(dbName string) error { + sql := fmt.Sprintf("create database %s precision 'ms'", dbName) + if d.Opts.Buffer > 0 { + sql += " buffer " + strconv.Itoa(d.Opts.Buffer) + } + if d.Opts.Pages > 0 { + sql += " pages " + strconv.Itoa(d.Opts.Pages) + } + if d.Opts.VGroups > 0 { + sql += " vgroups " + strconv.Itoa(d.Opts.VGroups) + } + if d.Opts.SttTrigger > 0 { + sql += " stt_trigger " + strconv.Itoa(d.Opts.SttTrigger) + } + if d.Opts.WalFsyncPeriod != nil { + sql += " wal_fsync_period " + strconv.Itoa(*d.Opts.WalFsyncPeriod) + } + if d.Opts.WalLevel != nil { + sql += " wal_level " + strconv.Itoa(*d.Opts.WalLevel) + } + if d.Opts.DBParameters != "" { + sql += " " + d.Opts.DBParameters + } + return async.GlobalAsync.TaosExecWithoutResult(d.Db.TaosConnection, sql) +} + +func (d *DbCreator) RemoveOldDB(dbName string) error { + sql := fmt.Sprintf("drop database %s", dbName) + return async.GlobalAsync.TaosExecWithoutResult(d.Db.TaosConnection, sql) +} + +func (d *DbCreator) Close() { + if d.Db != nil { + d.Db.Put() + } +} diff --git a/pkg/targets/tdengine/file_data_source.go b/pkg/targets/tdengine/file_data_source.go new file mode 100644 index 000000000..3a3530de2 --- /dev/null +++ b/pkg/targets/tdengine/file_data_source.go @@ -0,0 +1,65 @@ +package tdengine + +import ( + "bufio" + "strconv" + "strings" + + "github.com/timescale/tsbs/load" + "github.com/timescale/tsbs/pkg/data" + "github.com/timescale/tsbs/pkg/data/usecases/common" + "github.com/timescale/tsbs/pkg/targets" +) + +func newFileDataSource(fileName string) targets.DataSource { + br := load.GetBufferedReader(fileName) + + return &fileDataSource{scanner: bufio.NewScanner(br)} +} + +type fileDataSource struct { + scanner *bufio.Scanner + headers *common.GeneratedDataHeaders +} + +func (d *fileDataSource) Headers() *common.GeneratedDataHeaders { + return nil +} + +func (d *fileDataSource) NextItem() data.LoadedPoint { + ok := d.scanner.Scan() + if !ok && d.scanner.Err() == nil { // nothing scanned & no error = EOF + return data.LoadedPoint{} + } else if !ok { + fatal("scan error: %v", d.scanner.Err()) + return data.LoadedPoint{} + } + p := &point{} + line := d.scanner.Text() + p.sqlType = line[0] + switch line[0] { + case Insert: + parts := strings.SplitN(line, ",", 4) + p.subTable = parts[1] + p.fieldCount, _ = strconv.Atoi(parts[2]) + p.sql = strings.TrimSpace(parts[3]) + case CreateSTable: + parts := strings.SplitN(line, ",", 4) + p.superTable = parts[1] + p.subTable = parts[2] + p.sql = parts[3] + case CreateSubTable: + parts := strings.SplitN(line, ",", 4) + p.superTable = parts[1] + p.subTable = parts[2] + p.sql = parts[3][12:] + //case Modify: + // parts := strings.SplitN(line, ",", 4) + // p.superTable = parts[1] + // p.subTable = parts[2] + // p.sql = parts[3] + default: + panic(line) + } + return data.NewLoadedPoint(p) +} diff --git a/pkg/targets/tdengine/implemented_target.go b/pkg/targets/tdengine/implemented_target.go new file mode 100644 index 000000000..042a984dc --- /dev/null +++ b/pkg/targets/tdengine/implemented_target.go @@ -0,0 +1,58 @@ +package tdengine + +import ( + "bytes" + + "github.com/blagojts/viper" + "github.com/spf13/pflag" + "github.com/timescale/tsbs/pkg/data/serialize" + "github.com/timescale/tsbs/pkg/data/source" + "github.com/timescale/tsbs/pkg/targets" + "github.com/timescale/tsbs/pkg/targets/constants" +) + +func NewTarget() targets.ImplementedTarget { + return &tdengineTarget{} +} + +type tdengineTarget struct { +} + +func (t *tdengineTarget) TargetSpecificFlags(flagPrefix string, flagSet *pflag.FlagSet) { + TargetSpecificFlags(flagPrefix, flagSet) +} + +func TargetSpecificFlags(flagPrefix string, flagSet *pflag.FlagSet) { + flagSet.String(flagPrefix+"user", "root", "User to connect to TDengine") + flagSet.String(flagPrefix+"pass", "taosdata", "Password for user connecting to TDengine") + flagSet.String(flagPrefix+"host", "", "TDengine host") + flagSet.Int(flagPrefix+"port", 6030, "TDengine Port") + flagSet.Int(flagPrefix+"vgroups", 0, "TDengine DB vgroups") + flagSet.Int(flagPrefix+"pages", 0, "TDengine DB pages") + flagSet.Int(flagPrefix+"buffer", 0, "TDengine DB buffer") + flagSet.Int(flagPrefix+"stt_trigger", 0, "TDengine DB stt_trigger") + flagSet.Int(flagPrefix+"wal_fsync_period", 3000, "TDengine DB wal_fsync_period") + flagSet.Int(flagPrefix+"wal_level", 1, "TDengine DB wal_level") + flagSet.String(flagPrefix+"db_parameters", "", "TDengine DB parameters") +} + +func (t *tdengineTarget) TargetName() string { + return constants.FormatTDengine +} + +func (t *tdengineTarget) Serializer() serialize.PointSerializer { + return &Serializer{ + tableMap: map[string]struct{}{}, + superTable: map[string]*Table{}, + tmpBuf: &bytes.Buffer{}, + } +} + +func (t *tdengineTarget) Benchmark(targetDB string, dataSourceConfig *source.DataSourceConfig, v *viper.Viper, +) (targets.Benchmark, error) { + var loadingOptions LoadingOptions + if err := v.Unmarshal(&loadingOptions); err != nil { + return nil, err + } + return NewBenchmark(targetDB, &loadingOptions, dataSourceConfig) +} diff --git a/pkg/targets/tdengine/process.go b/pkg/targets/tdengine/process.go new file mode 100644 index 000000000..64def336a --- /dev/null +++ b/pkg/targets/tdengine/process.go @@ -0,0 +1,225 @@ +package tdengine + +import ( + "bytes" + "context" + "fmt" + "io/ioutil" + "sync" + + "github.com/timescale/tsbs/pkg/targets" + "github.com/timescale/tsbs/pkg/targets/tdengine/async" + "github.com/timescale/tsbs/pkg/targets/tdengine/commonpool" +) + +type syncCSI struct { + m sync.Map //table:ctx +} + +const Size1M = 1 * 1024 * 1024 + +type Ctx struct { + c context.Context + cancel context.CancelFunc +} + +var globalSCI = &syncCSI{} + +type processor struct { + opts *LoadingOptions + dbName string + sci *syncCSI + _db *commonpool.Conn + wg *sync.WaitGroup + buf *bytes.Buffer +} + +func newProcessor(opts *LoadingOptions, dbName string) *processor { + return &processor{opts: opts, dbName: dbName, sci: globalSCI, wg: &sync.WaitGroup{}, buf: &bytes.Buffer{}} +} + +func (p *processor) Init(_ int, doLoad, _ bool) { + if !doLoad { + return + } + p.buf.Grow(Size1M) + var err error + p._db, err = commonpool.GetConnection(p.opts.User, p.opts.Pass, p.opts.Host, p.opts.Port) + if err != nil { + panic(err) + } + err = async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, "use "+p.dbName) + if err != nil { + panic(err) + } +} + +func (p *processor) ProcessBatch(b targets.Batch, doLoad bool) (metricCount, rowCount uint64) { + batches := b.(*hypertableArr) + rowCnt := uint64(0) + metricCnt := batches.totalMetric + if !doLoad { + for _, sqls := range batches.m { + rowCnt += uint64(len(sqls)) + } + return metricCnt, rowCnt + } + p.buf.Reset() + p.buf.WriteString("create table") + for _, row := range batches.createSql { + switch row.sqlType { + case CreateSTable: + c, cancel := context.WithCancel(context.Background()) + ctx := &Ctx{ + c: c, + cancel: cancel, + } + actual, _ := p.sci.m.LoadOrStore(row.superTable, ctx) + err := async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, row.sql) + if err != nil { + fmt.Println(row.sql) + panic(err) + } + GlobalTable.Store(row.subTable, nothing) + actual.(*Ctx).cancel() + case CreateSubTable: + c, cancel := context.WithCancel(context.Background()) + ctx := &Ctx{ + c: c, + cancel: cancel, + } + actual, _ := p.sci.m.LoadOrStore(row.subTable, ctx) + + //check super table created + _, ok := GlobalTable.Load(row.superTable) + if !ok { + v, ok := p.sci.m.Load(row.superTable) + if ok { + <-v.(*Ctx).c.Done() + if p.buf.Len()+len(row.sql) > Size1M { + sql := p.buf.String() + err := async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, sql) + if err != nil { + fmt.Println(sql) + panic(err) + } + p.buf.Reset() + p.buf.WriteString("create table") + } + p.buf.WriteString(row.sql) + GlobalTable.Store(row.subTable, nothing) + actual.(*Ctx).cancel() + continue + } + // wait for super table created + superTableC, superTableCancel := context.WithCancel(context.Background()) + superTableCtx := &Ctx{ + c: superTableC, + cancel: superTableCancel, + } + superTableActual, _ := p.sci.m.LoadOrStore(row.superTable, superTableCtx) + <-superTableActual.(*Ctx).c.Done() + } + if p.buf.Len()+len(row.sql) > Size1M { + sql := p.buf.String() + err := async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, sql) + if err != nil { + fmt.Println(sql) + panic(err) + } + p.buf.Reset() + p.buf.WriteString("create table") + } + p.buf.WriteString(row.sql) + GlobalTable.Store(row.subTable, nothing) + actual.(*Ctx).cancel() + default: + panic("impossible") + } + } + if p.buf.Len() > 12 { + sql := p.buf.String() + err := async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, sql) + if err != nil { + fmt.Println(sql) + panic(err) + } + } + p.buf.Reset() + p.wg.Add(len(batches.m)) + for tableName := range batches.m { + tableName := tableName + go func() { + defer p.wg.Done() + _, ok := GlobalTable.Load(tableName) + if ok { + return + } + v, ok := p.sci.m.Load(tableName) + if ok { + <-v.(*Ctx).c.Done() + return + } + c, cancel := context.WithCancel(context.Background()) + ctx := &Ctx{ + c: c, + cancel: cancel, + } + actual, _ := p.sci.m.LoadOrStore(tableName, ctx) + <-actual.(*Ctx).c.Done() + return + }() + } + p.wg.Wait() + p.buf.WriteString("insert into ") + for tableName, sqls := range batches.m { + rowCnt += uint64(len(sqls)) + if p.buf.Len()+len(sqls[0])+len(tableName)+7 > Size1M { + sql := p.buf.String() + err := async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, sql) + if err != nil { + ioutil.WriteFile("wrongsql.txt", []byte(sql), 0755) + fmt.Println(sql) + panic(err) + } + p.buf.Reset() + p.buf.WriteString("insert into ") + } + p.buf.WriteString(tableName) + p.buf.WriteString(" values") + for i := 0; i < len(sqls); i++ { + if p.buf.Len()+len(sqls[i]) > Size1M { + sql := p.buf.String() + err := async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, sql) + if err != nil { + ioutil.WriteFile("wrongsql.txt", []byte(sql), 0755) + fmt.Println(sql) + panic(err) + } + p.buf.Reset() + p.buf.WriteString("insert into ") + p.buf.WriteString(tableName) + p.buf.WriteString(" values") + } + p.buf.WriteString(sqls[i]) + } + } + if p.buf.Len() > 0 { + sql := p.buf.String() + err := async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, sql) + if err != nil { + fmt.Println(sql) + panic(err) + } + p.buf.Reset() + } + + batches.Reset() + return metricCnt, rowCnt +} + +func (p *processor) Close(doLoad bool) { + if doLoad { + p._db.Put() + } +} diff --git a/pkg/targets/tdengine/program_options.go b/pkg/targets/tdengine/program_options.go new file mode 100644 index 000000000..164a5ccab --- /dev/null +++ b/pkg/targets/tdengine/program_options.go @@ -0,0 +1,15 @@ +package tdengine + +type LoadingOptions struct { + User string + Pass string + Host string + Port int + VGroups int + Buffer int + Pages int + SttTrigger int + WalFsyncPeriod *int + WalLevel *int + DBParameters string +} diff --git a/pkg/targets/tdengine/scan.go b/pkg/targets/tdengine/scan.go new file mode 100644 index 000000000..3789dddf3 --- /dev/null +++ b/pkg/targets/tdengine/scan.go @@ -0,0 +1,95 @@ +package tdengine + +import ( + "bytes" + "sync" + + "github.com/spaolacci/murmur3" + "github.com/timescale/tsbs/pkg/data" + "github.com/timescale/tsbs/pkg/targets" +) + +// indexer is used to consistently send the same hostnames to the same worker +type indexer struct { + buffer *bytes.Buffer + prefix []byte + partitions int + hashEndGroups []uint32 + tmp map[string]uint +} + +func (i *indexer) GetIndex(item data.LoadedPoint) uint { + p := item.Data.(*point) + idx, exist := i.tmp[p.subTable] + if exist { + return idx + } + i.buffer.Write(i.prefix) + i.buffer.WriteString(p.subTable) + hash := murmur3.Sum32WithSeed(i.buffer.Bytes(), 0x12345678) + i.buffer.Reset() + for j := 0; j < i.partitions; j++ { + if hash <= i.hashEndGroups[j] { + idx = uint(j) + break + } + } + i.tmp[p.subTable] = idx + return idx +} + +// point is a single row of data keyed by which superTable it belongs +type point struct { + sqlType byte + superTable string + subTable string + fieldCount int + sql string +} + +var GlobalTable = sync.Map{} + +type hypertableArr struct { + createSql []*point + m map[string][]string + totalMetric uint64 + cnt uint +} + +func (ha *hypertableArr) Len() uint { + return ha.cnt +} + +func (ha *hypertableArr) Append(item data.LoadedPoint) { + that := item.Data.(*point) + if that.sqlType == Insert { + ha.m[that.subTable] = append(ha.m[that.subTable], that.sql) + ha.totalMetric += uint64(that.fieldCount) + ha.cnt++ + } else { + ha.createSql = append(ha.createSql, that) + } +} + +func (ha *hypertableArr) Reset() { + ha.m = map[string][]string{} + ha.cnt = 0 + ha.createSql = ha.createSql[:0] +} + +type BatchFactory struct { + pool *sync.Pool +} + +func (b *BatchFactory) New() targets.Batch { + return b.pool.Get().(*hypertableArr) +} + +func NewBatchFactory() *BatchFactory { + pool := &sync.Pool{New: func() interface{} { + return &hypertableArr{ + m: map[string][]string{}, + } + }} + return &BatchFactory{pool: pool} +} diff --git a/pkg/targets/tdengine/serializer.go b/pkg/targets/tdengine/serializer.go new file mode 100644 index 000000000..0e248a1de --- /dev/null +++ b/pkg/targets/tdengine/serializer.go @@ -0,0 +1,232 @@ +package tdengine + +import ( + "bytes" + "fmt" + "io" + "strconv" + "strings" + + "github.com/timescale/tsbs/pkg/data" +) + +type Serializer struct { + tmpBuf *bytes.Buffer + tableMap map[string]struct{} + superTable map[string]*Table +} + +var nothing = struct{}{} + +type Table struct { + columns map[string]struct{} + tags map[string]struct{} +} + +func FastFormat(buf *bytes.Buffer, v interface{}) string { + switch v.(type) { + case int: + buf.WriteString(strconv.Itoa(v.(int))) + return "bigint" + case int64: + buf.WriteString(strconv.FormatInt(v.(int64), 10)) + return "bigint" + case float64: + buf.WriteString(strconv.FormatFloat(v.(float64), 'f', -1, 64)) + return "double" + case float32: + buf.WriteString(strconv.FormatFloat(float64(v.(float32)), 'f', -1, 32)) + return "double" + case bool: + buf.WriteString(strconv.FormatBool(v.(bool))) + return "bool" + case []byte: + buf.WriteByte('\'') + buf.WriteString(string(v.([]byte))) + buf.WriteByte('\'') + return "binary(30)" + case string: + buf.WriteByte('\'') + buf.WriteString(v.(string)) + buf.WriteByte('\'') + return "binary(30)" + case nil: + buf.WriteString("null") + return "null" + default: + panic(fmt.Sprintf("unknown field type for %#v", v)) + } +} + +var tmpMD5 = map[string]string{} +var tmpIndex = 0 + +func calculateTable(src []byte) string { + key := string(src) + v, exist := tmpMD5[key] + if exist { + return v + } + tmpIndex += 1 + v = fmt.Sprintf("t_%d", tmpIndex) + tmpMD5[key] = v + return v +} + +const ( + Insert = '1' + CreateSTable = '2' + CreateSubTable = '3' + Modify = '4' +) + +type tbNameRule struct { + tag string + prefix string + nilValue string +} + +var tbRuleMap = map[string]*tbNameRule{ + "cpu": { + tag: "hostname", + nilValue: "host_null", + }, + "readings": { + tag: "name", + prefix: "r_", + nilValue: "r_truck_null", + }, + "diagnostics": { + tag: "name", + prefix: "d_", + nilValue: "d_truck_null", + }, +} + +func (s *Serializer) Serialize(p *data.Point, w io.Writer) error { + var fieldKeys []string + var fieldValues []string + var fieldTypes []string + var tagValues []string + var tagKeys []string + var tagTypes []string + tKeys := p.TagKeys() + tValues := p.TagValues() + fKeys := p.FieldKeys() + fValues := p.FieldValues() + superTable := string(p.MeasurementName()) + for i, value := range fValues { + fType := FastFormat(s.tmpBuf, value) + fieldKeys = append(fieldKeys, convertKeywords(string(fKeys[i]))) + fieldTypes = append(fieldTypes, fType) + fieldValues = append(fieldValues, s.tmpBuf.String()) + s.tmpBuf.Reset() + } + + rule := tbRuleMap[superTable] + fixedName := "" + for i, value := range tValues { + tType := FastFormat(s.tmpBuf, value) + if rule != nil && len(fixedName) == 0 && string(tKeys[i]) == rule.tag { + str, is := value.(string) + if is { + fixedName = str + } + } + tagKeys = append(tagKeys, convertKeywords(string(tKeys[i]))) + tagTypes = append(tagTypes, tType) + tagValues = append(tagValues, s.tmpBuf.String()) + s.tmpBuf.Reset() + } + + subTable := "" + if rule != nil { + if len(fixedName) != 0 { + if len(rule.prefix) == 0 { + subTable = fixedName + } else { + s.tmpBuf.WriteString(rule.prefix) + s.tmpBuf.WriteString(fixedName) + subTable = s.tmpBuf.String() + s.tmpBuf.Reset() + } + } else { + subTable = rule.nilValue + } + } else { + s.tmpBuf.WriteString(superTable) + for i, v := range tagValues { + s.tmpBuf.WriteByte(',') + s.tmpBuf.WriteString(tagKeys[i]) + s.tmpBuf.WriteByte('=') + s.tmpBuf.WriteString(v) + } + subTable = calculateTable(s.tmpBuf.Bytes()) + s.tmpBuf.Reset() + } + stable, exist := s.superTable[superTable] + if !exist { + for i := 0; i < len(fieldTypes); i++ { + s.tmpBuf.WriteByte(',') + s.tmpBuf.WriteString(fieldKeys[i]) + s.tmpBuf.WriteByte(' ') + s.tmpBuf.WriteString(fieldTypes[i]) + } + fieldStr := s.tmpBuf.String() + s.tmpBuf.Reset() + for i := 0; i < len(tagTypes); i++ { + s.tmpBuf.WriteString(tagKeys[i]) + s.tmpBuf.WriteByte(' ') + s.tmpBuf.WriteString(tagTypes[i]) + if i != len(tagTypes)-1 { + s.tmpBuf.WriteByte(',') + } + } + tagStr := s.tmpBuf.String() + fmt.Fprintf(w, "%c,%s,%s,create table %s (ts timestamp%s) tags (%s)\n", CreateSTable, superTable, subTable, superTable, fieldStr, tagStr) + s.tmpBuf.Reset() + table := &Table{ + columns: map[string]struct{}{}, + tags: map[string]struct{}{}, + } + for _, key := range fieldKeys { + table.columns[key] = nothing + } + for _, key := range tagKeys { + table.tags[key] = nothing + } + s.superTable[superTable] = table + } else { + for _, key := range fieldKeys { + if _, exist = stable.columns[key]; !exist { + fmt.Fprintf(w, "%c,%s,%s,alter table %s add COLUMN %s double\n", Modify, superTable, subTable, superTable, key) + stable.columns[key] = nothing + } + } + for _, key := range tagKeys { + if _, exist = stable.tags[key]; !exist { + fmt.Fprintf(w, "%c,%s,%s,alter table %s add TAG %s binary(30)\n", Modify, superTable, subTable, superTable, key) + stable.tags[key] = nothing + } + } + } + _, exist = s.tableMap[subTable] + if !exist { + fmt.Fprintf(w, "%c,%s,%s,create table %s using %s (%s) tags (%s)\n", CreateSubTable, superTable, subTable, subTable, superTable, strings.Join(tagKeys, ","), strings.Join(tagValues, ",")) + s.tableMap[subTable] = nothing + } + + fmt.Fprintf(w, "%c,%s,%d,(%d,%s)\n", Insert, subTable, len(fieldValues), p.TimestampInUnixMs(), strings.Join(fieldValues, ",")) + return nil +} + +var keyWords = map[string]bool{ + "port": true, +} + +func convertKeywords(s string) string { + if is := keyWords[s]; is { + return fmt.Sprintf("`%s`", s) + } + return s +} diff --git a/pkg/targets/tdengine/thread/locker.go b/pkg/targets/tdengine/thread/locker.go new file mode 100644 index 000000000..2ad485075 --- /dev/null +++ b/pkg/targets/tdengine/thread/locker.go @@ -0,0 +1,33 @@ +package thread + +import "runtime" + +type Locker struct { + c chan struct{} +} + +func NewLocker(count int) *Locker { + return &Locker{c: make(chan struct{}, count)} +} + +func (l *Locker) Lock() { + l.c <- struct{}{} +} + +func (l *Locker) Unlock() { + <-l.c +} + +var c chan struct{} + +func Lock() { + c <- struct{}{} +} + +func Unlock() { + <-c +} + +func init() { + c = make(chan struct{}, runtime.NumCPU()) +} diff --git a/pkg/targets/tdengine/thread/locker_test.go b/pkg/targets/tdengine/thread/locker_test.go new file mode 100644 index 000000000..f49be5c33 --- /dev/null +++ b/pkg/targets/tdengine/thread/locker_test.go @@ -0,0 +1,41 @@ +package thread + +import ( + "testing" +) + +// @author: xftan +// @date: 2021/12/14 15:16 +// @description: test NewLocker +func TestNewLocker(t *testing.T) { + type args struct { + count int + } + tests := []struct { + name string + args args + }{ + { + name: "test", + args: args{ + count: 1, + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + locker := NewLocker(tt.args.count) + locker.Lock() + locker.Unlock() + }) + } +} + +// @author: xftan +// @date: 2021/12/14 15:16 +// @description: test DefaultLocker +func TestDefaultLocker(t *testing.T) { + Lock() + t.Log("success") + defer Unlock() +} diff --git a/pkg/targets/tdenginestmt2/benchmark.go b/pkg/targets/tdenginestmt2/benchmark.go new file mode 100644 index 000000000..2038b34ae --- /dev/null +++ b/pkg/targets/tdenginestmt2/benchmark.go @@ -0,0 +1,105 @@ +package tdenginestmt2 + +import "C" +import ( + "math" + + "github.com/timescale/tsbs/pkg/data/source" + "github.com/timescale/tsbs/pkg/targets" + "github.com/timescale/tsbs/pkg/targets/tdengine" +) + +func NewBenchmark(dbName string, opts *tdengine.LoadingOptions, dataSourceConfig *source.DataSourceConfig) (targets.Benchmark, error) { + var ds targets.DataSource + if dataSourceConfig.Type == source.FileDataSourceType { + ds = newFileDataSource(dataSourceConfig.File.Location) + useCase, scale := ds.(*fileDataSource).readHeader() + return &benchmark{ + opts: opts, + dataSource: ds, + dbName: dbName, + factory: NewBatchFactory(), + useCase: useCase, + scale: scale, + }, nil + } else { + panic("not implement") + } +} + +type benchmark struct { + opts *tdengine.LoadingOptions + dataSource targets.DataSource + dbName string + batchSize uint + factory targets.BatchFactory + indexer targets.PointIndexer + useCase byte + scale uint32 + tableOffset [3][]uint32 + partitionTables [3][][]uint32 +} + +func (b *benchmark) GetDataSource() targets.DataSource { + return b.dataSource +} + +func (b *benchmark) GetBatchFactory() targets.BatchFactory { + return b.factory +} + +func (b *benchmark) GetPointIndexer(uint) targets.PointIndexer { + return b.indexer +} + +func (b *benchmark) GetProcessor() targets.Processor { + return newProcessor(b.opts, b.dbName, b.batchSize, b.useCase, b.scale, b.partitionTables, b.tableOffset) +} + +func (b *benchmark) GetDBCreator() targets.DBCreator { + return &DbCreator{ + DbCreator: tdengine.DbCreator{Opts: b.opts}, + useCase: b.useCase, + ds: b.dataSource.(*fileDataSource), + } +} + +func (b *benchmark) SetConfig(batchSize uint, workers uint) { + b.batchSize = batchSize + + b.dataSource.(*fileDataSource).SetConfig(int(workers), int(batchSize), int(b.scale)) + factory := b.factory.(*BatchFactory) + factory.batchSize = batchSize + if workers > 1 { + interval := uint32(math.MaxUint32 / workers) + hashEndGroups := make([]uint32, workers) + for i := 0; i < int(workers); i++ { + if i == int(workers)-1 { + hashEndGroups[i] = math.MaxUint32 + } else { + hashEndGroups[i] = interval*uint32(i+1) - 1 + } + } + prefix := []byte("1." + b.dbName + ".") + idx, tableOffset, hostTableIndex, readingsTableIndex, diagnosticsTableIndex := NewIndexer(prefix, int(workers), hashEndGroups, b.useCase, b.scale) + b.tableOffset = tableOffset + b.partitionTables = [3][][]uint32{hostTableIndex, readingsTableIndex, diagnosticsTableIndex} + b.indexer = idx + } else { + b.indexer = &targets.ConstantIndexer{} + tmp := make([]uint32, b.scale+1) + for i := uint32(0); i < b.scale+1; i++ { + tmp[i] = i + } + switch b.useCase { + case CpuCase: + b.partitionTables[SuperTableHost] = [][]uint32{tmp} + b.tableOffset[SuperTableHost] = tmp + case IoTCase: + b.partitionTables[SuperTableReadings] = [][]uint32{tmp} + b.tableOffset[SuperTableReadings] = tmp + b.partitionTables[SuperTableDiagnostics] = [][]uint32{tmp} + b.tableOffset[SuperTableDiagnostics] = tmp + } + } +} diff --git a/pkg/targets/tdenginestmt2/cb.go b/pkg/targets/tdenginestmt2/cb.go new file mode 100644 index 000000000..62a3f57bf --- /dev/null +++ b/pkg/targets/tdenginestmt2/cb.go @@ -0,0 +1,23 @@ +package tdenginestmt2 + +/* +#cgo CFLAGS: -I/usr/include +#cgo linux LDFLAGS: -L/usr/lib -ltaos +#include +#include +#include +#include +*/ +import "C" +import ( + "unsafe" + + "github.com/taosdata/driver-go/v3/wrapper" + "github.com/taosdata/driver-go/v3/wrapper/cgo" +) + +//export QueryCallback2 +func QueryCallback2(p unsafe.Pointer, res *C.TAOS_RES, code C.int) { + caller := (*(*cgo.Handle)(p)).Value().(wrapper.Caller) + caller.QueryCall(unsafe.Pointer(res), int(code)) +} diff --git a/pkg/targets/tdenginestmt2/creator.go b/pkg/targets/tdenginestmt2/creator.go new file mode 100644 index 000000000..64ff3aa2e --- /dev/null +++ b/pkg/targets/tdenginestmt2/creator.go @@ -0,0 +1,36 @@ +package tdenginestmt2 + +import ( + "github.com/timescale/tsbs/pkg/targets/tdengine" + "github.com/timescale/tsbs/pkg/targets/tdengine/async" +) + +type DbCreator struct { + tdengine.DbCreator + useCase byte + ds *fileDataSource +} + +func (d *DbCreator) PostCreateDB(dbName string) error { + err := async.GlobalAsync.TaosExecWithoutResult(d.Db.TaosConnection, "use "+dbName) + if err != nil { + return err + } + switch d.useCase { + case CpuCase: + err = async.GlobalAsync.TaosExecWithoutResult(d.Db.TaosConnection, CpuSql) + if err != nil { + return err + } + case IoTCase: + err = async.GlobalAsync.TaosExecWithoutResult(d.Db.TaosConnection, ReadingsSql) + if err != nil { + return err + } + err = async.GlobalAsync.TaosExecWithoutResult(d.Db.TaosConnection, DiagnosticsSql) + if err != nil { + return err + } + } + return nil +} diff --git a/pkg/targets/tdenginestmt2/file_data_source.go b/pkg/targets/tdenginestmt2/file_data_source.go new file mode 100644 index 000000000..f38795d42 --- /dev/null +++ b/pkg/targets/tdenginestmt2/file_data_source.go @@ -0,0 +1,198 @@ +package tdenginestmt2 + +import ( + "bufio" + "encoding/binary" + "io" + "log" + "os" + "reflect" + "unsafe" + + "github.com/timescale/tsbs/pkg/data" + "github.com/timescale/tsbs/pkg/data/usecases/common" + "github.com/timescale/tsbs/pkg/targets" +) + +var fatal = log.Fatalf + +func newFileDataSource(fileName string) targets.DataSource { + br := GetBufferedReader(fileName) + return &fileDataSource{br: br, exchange: make(chan struct{}, 1), exchangeStatus: make(chan int, 1)} +} + +const ( + defaultReadSize = 4 << 20 // 4 MB +) + +func GetBufferedReader(fileName string) *bufio.Reader { + if len(fileName) == 0 { + // Read from STDIN + return bufio.NewReaderSize(os.Stdin, defaultReadSize) + } + // Read from specified file + file, err := os.Open(fileName) + if err != nil { + fatal("cannot open file for read %s: %v", fileName, err) + return nil + } + return bufio.NewReaderSize(file, defaultReadSize) +} + +type fileDataSource struct { + br *bufio.Reader + cache []*[]byte + cacheIndex int + cacheSize int + scale int + exchange chan struct{} + exchangeStatus chan int + readDirect bool +} + +/* + fixed header + +| version(1 byte) | case (1 byte) | scale (4 bytes) + +*/ + +func (d *fileDataSource) readHeader() (byte, uint32) { + buf := make([]byte, 6) + _, err := io.ReadFull(d.br, buf) + if err != nil { + fatal("cannot read header: %v", err) + } + if buf[0] != 1 { + fatal("invalid version: %d", buf[0]) + } + scale := binary.LittleEndian.Uint32(buf[2:]) + d.scale = int(scale) + return buf[1], scale +} + +func (d *fileDataSource) SetConfig(worker int, batchSize int, scale int) { + globalSlicePool = NewSlicePool(worker, batchSize, scale) +} + +func (d *fileDataSource) Headers() *common.GeneratedDataHeaders { + return nil +} + +var globalLoadedPoint = data.NewLoadedPoint(nil) + +type SlicePool struct { + smallPool chan []*[]byte +} + +var globalSlicePool *SlicePool + +func NewSlicePool(worker, batchSize, scale int) *SlicePool { + totalItem := (worker*batchSize)*500 + scale*2 + 2 + arrayLen := worker * 2 + pool := make(chan []*[]byte, arrayLen*50) + totalBytes := make([]*[]byte, totalItem) + for i := 0; i < totalItem; i++ { + bs := make([]byte, 0, 256) + totalBytes[i] = &bs + } + bss := SplitBytes(totalBytes, arrayLen) + for i := 0; i < arrayLen; i++ { + pool <- bss[i] + } + return &SlicePool{smallPool: pool} +} + +func (p *SlicePool) Get() []*[]byte { + //return <-p.smallPool + select { + case b := <-p.smallPool: + return b + default: + panic("no more slice") + } +} + +func (p *SlicePool) Put(b []*[]byte) { + select { + case p.smallPool <- b: + default: + return + } +} + +func (d *fileDataSource) NextItem() data.LoadedPoint { + if d.cacheIndex == d.cacheSize { + d.cache = globalSlicePool.Get() + d.cacheSize = len(d.cache) + d.cacheIndex = 0 + } + u8length, err := d.br.ReadByte() + if err != nil { + if err == io.EOF { + globalLoadedPoint.Data = nil + return globalLoadedPoint + } + panic(err) + } + if u8length < 128 { + length := int(u8length) + ptr := d.cache[d.cacheIndex] + (*reflect.SliceHeader)(unsafe.Pointer(ptr)).Len = length + s := *ptr + nn, err := d.br.Read(s) + if err != nil { + panic(err) + } + if nn != length { + n := nn + for n < length && err == nil { + nn, err = d.br.Read(s[n:]) + n += nn + } + } + d.cacheIndex++ + globalLoadedPoint.Data = ptr + return globalLoadedPoint + } else { + tmp, err := d.br.ReadByte() + if err != nil { + panic(err) + } + u16Length := int(u8length&0x7f) + int(tmp<<7) + ptr := d.cache[d.cacheIndex] + (*reflect.SliceHeader)(unsafe.Pointer(ptr)).Len = u16Length + s := *ptr + nn, err := d.br.Read(s) + if err != nil { + panic(err) + } + if nn != u16Length { + n := nn + for n < u16Length && err == nil { + nn, err = d.br.Read(s[n:]) + n += nn + } + } + d.cacheIndex++ + globalLoadedPoint.Data = ptr + return globalLoadedPoint + } +} + +func SplitBytes(arr []*[]byte, n int) [][]*[]byte { + if n <= 0 { + return nil + } + subArraySize := (len(arr) + n - 1) / n + result := make([][]*[]byte, 0, n) + for i := 0; i < len(arr); i += subArraySize { + end := i + subArraySize + if end > len(arr) { + end = len(arr) + } + result = append(result, arr[i:end]) + } + + return result +} diff --git a/pkg/targets/tdenginestmt2/implemented_target.go b/pkg/targets/tdenginestmt2/implemented_target.go new file mode 100644 index 000000000..68d4b3fa8 --- /dev/null +++ b/pkg/targets/tdenginestmt2/implemented_target.go @@ -0,0 +1,44 @@ +package tdenginestmt2 + +import ( + "bytes" + + "github.com/blagojts/viper" + "github.com/spf13/pflag" + "github.com/timescale/tsbs/pkg/data/serialize" + "github.com/timescale/tsbs/pkg/data/source" + "github.com/timescale/tsbs/pkg/targets" + "github.com/timescale/tsbs/pkg/targets/constants" + "github.com/timescale/tsbs/pkg/targets/tdengine" +) + +func NewTarget() targets.ImplementedTarget { + return &tdengineStmt2Target{} +} + +type tdengineStmt2Target struct { +} + +func (t *tdengineStmt2Target) TargetSpecificFlags(flagPrefix string, flagSet *pflag.FlagSet) { + tdengine.TargetSpecificFlags(flagPrefix, flagSet) +} + +func (t *tdengineStmt2Target) TargetName() string { + return constants.FormatTDengineStmt2 +} + +func (t *tdengineStmt2Target) Serializer() serialize.PointSerializer { + return &Serializer{ + tmpBuf: &bytes.Buffer{}, + writeBuf: &bytes.Buffer{}, + } +} + +func (t *tdengineStmt2Target) Benchmark(targetDB string, dataSourceConfig *source.DataSourceConfig, v *viper.Viper, +) (targets.Benchmark, error) { + var loadingOptions tdengine.LoadingOptions + if err := v.Unmarshal(&loadingOptions); err != nil { + return nil, err + } + return NewBenchmark(targetDB, &loadingOptions, dataSourceConfig) +} diff --git a/pkg/targets/tdenginestmt2/process.go b/pkg/targets/tdenginestmt2/process.go new file mode 100644 index 000000000..16ded2f93 --- /dev/null +++ b/pkg/targets/tdenginestmt2/process.go @@ -0,0 +1,932 @@ +package tdenginestmt2 + +/* +#cgo CFLAGS: -I/usr/include +#cgo linux LDFLAGS: -L/usr/lib -ltaos +#include +#include +#include +#include +extern void QueryCallback2(void *param,TAOS_RES *,int code); +void taos_query_a_wrapper2(TAOS *taos,const char *sql, void *param){ + return taos_query_a(taos,sql,QueryCallback2,param); +}; +*/ +import "C" +import ( + "context" + "fmt" + "reflect" + "strconv" + "sync" + "sync/atomic" + "unsafe" + + taosCommon "github.com/taosdata/driver-go/v3/common" + "github.com/taosdata/driver-go/v3/wrapper" + "github.com/taosdata/driver-go/v3/wrapper/cgo" + "github.com/timescale/tsbs/pkg/targets" + "github.com/timescale/tsbs/pkg/targets/tdengine" + "github.com/timescale/tsbs/pkg/targets/tdengine/async" + "github.com/timescale/tsbs/pkg/targets/tdengine/commonpool" +) + +type Ctx struct { + c context.Context + cancel context.CancelFunc +} + +type processor struct { + tableSlot [3][]uint32 + cBuffers [3]*bufferPointer + tableNamePointerCache [3][]unsafe.Pointer + stmt2CHandle [3]unsafe.Pointer + stmt2CBHandle [3]*async.Handler + hostTableNameCBuffer unsafe.Pointer + readingsTableNameCBuffer unsafe.Pointer + diagnosticsTableNameCBuffer unsafe.Pointer + + hostSlot [][][]byte + readingsSlot [][][]byte + diagnosticsSlot [][][]byte + hostBatchIndexer []uint32 + readingsBatchIndexer []uint32 + diagnosticsBatchIndexer []uint32 + wg *sync.WaitGroup + partitionTable [3][][]uint32 + + _db *commonpool.Conn + bufIndex int + cBufPosition unsafe.Pointer + cBuf unsafe.Pointer + caller *async.Caller + hp unsafe.Pointer + handle cgo.Handle + + hostExecCount int32 + readingsExecCount int32 + diagnosticsExecCount int32 + createWg *sync.WaitGroup + opts *tdengine.LoadingOptions + dbName string + useCase int + batchSize uint + scale uint32 + + exitSign chan struct{} + finishWg *sync.WaitGroup +} + +type bufferPointer struct { + bindVP unsafe.Pointer + tableNameP []unsafe.Pointer + bindsP [][]unsafe.Pointer + colP []unsafe.Pointer + isNullP []unsafe.Pointer +} + +const Size1M = 1 << 20 + +func newProcessor(opts *tdengine.LoadingOptions, dbName string, batchSize uint, useCase byte, scale uint32, partitionTable [3][][]uint32, tableOffset [3][]uint32) *processor { + p := &processor{ + tableSlot: tableOffset, + partitionTable: partitionTable, + opts: opts, + dbName: dbName, + batchSize: batchSize, + wg: &sync.WaitGroup{}, + createWg: &sync.WaitGroup{}, + useCase: int(useCase), + scale: scale, + exitSign: make(chan struct{}, 1), + finishWg: &sync.WaitGroup{}, + } + return p +} + +const ( + CpuHandleIndex = 0 + ReadingsHandleIndex = 1 + DiagnosticsHandleIndex = 2 +) + +func (p *processor) Init(id int, doLoad, _ bool) { + if !doLoad { + return + } + var err error + p._db, err = commonpool.GetConnection(p.opts.User, p.opts.Pass, p.opts.Host, p.opts.Port) + if err != nil { + panic(err) + } + err = async.GlobalAsync.TaosExecWithoutResult(p._db.TaosConnection, "use "+p.dbName) + if err != nil { + panic(err) + } + defer func() { + if err := recover(); err != nil { + p.Close(true) + panic(err) + } + }() + p.caller = async.NewCaller() + p.handle = cgo.NewHandle(p.caller) + p.hp = p.handle.Pointer() + p.cBuf = C.malloc(Size1M) + createTable := []byte("create table") + createTableP := unsafe.Pointer(&createTable[0]) + C.memcpy(p.cBuf, createTableP, C.size_t(12)) + p.cBufPosition = p.cBuf + p.bufIndex = 0 + + // max table name 23 + switch p.useCase { + case CpuCase: + // init stmt2 + handler := async.GlobalAsync.HandlerPool.Get() + prepareSql := "insert into ? values(?,?,?,?,?,?,?,?,?,?,?)" + stmt2Pointer := wrapper.TaosStmt2Init(p._db.TaosConnection, int64(1<<56|id), true, true, handler.Handler) + if stmt2Pointer == nil { + panic(fmt.Errorf("failed to create stmt2")) + } + code := wrapper.TaosStmt2Prepare(stmt2Pointer, prepareSql) + if code != 0 { + errMsg := wrapper.TaosStmt2Error(stmt2Pointer) + panic(fmt.Errorf("failed to prepare stmt2: %s", errMsg)) + } + + // init table name c buffer + hostTableIndex := p.partitionTable[SuperTableHost][id] + p.tableNamePointerCache[SuperTableHost] = make([]unsafe.Pointer, len(hostTableIndex)) + p.tableNamePointerCache[SuperTableHost] = make([]unsafe.Pointer, len(hostTableIndex)) + p.hostTableNameCBuffer = C.calloc(C.size_t(24*(len(hostTableIndex))), 1) + cursor := p.hostTableNameCBuffer + prefix := []byte("host_") + var tableNameBytes []byte + for i := uint64(0); i < uint64(len(hostTableIndex)); i++ { + if hostTableIndex[i] == 0 { + tableNameNull := []byte("host_null") + C.memcpy(cursor, unsafe.Pointer(&tableNameNull[0]), C.size_t(len(tableNameNull))) + } else { + tableNameBytes = strconv.AppendUint(prefix, uint64(hostTableIndex[i]-1), 10) + C.memcpy(cursor, unsafe.Pointer(&tableNameBytes[0]), C.size_t(len(tableNameBytes))) + } + p.tableNamePointerCache[SuperTableHost][i] = cursor + cursor = unsafe.Pointer(uintptr(cursor) + 24) + } + + // crate stmt2 bind buffer + cBuffer := allocBuffer(len(hostTableIndex), int(p.batchSize), []int{ + 8, + 8, + 8, + 8, + 8, + 8, + 8, + 8, + 8, + 8, + 8, + }, []int{ + taosCommon.TSDB_DATA_TYPE_TIMESTAMP, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + taosCommon.TSDB_DATA_TYPE_BIGINT, + }) + p.cBuffers[CpuHandleIndex] = cBuffer + p.stmt2CHandle[CpuHandleIndex] = stmt2Pointer + p.stmt2CBHandle[CpuHandleIndex] = handler + + // init slot + p.hostSlot = make([][][]byte, len(hostTableIndex)) + for i := 0; i < len(hostTableIndex); i++ { + p.hostSlot[i] = make([][]byte, 0, 10) + } + p.hostBatchIndexer = make([]uint32, 0, p.batchSize) + p.receiveHostExecResult() + + case IoTCase: + // init stmt2 + handlerReading := async.GlobalAsync.HandlerPool.Get() + prepareReadingSql := "insert into ? values(?,?,?,?,?,?,?,?)" + stmt2ReadingPointer := wrapper.TaosStmt2Init(p._db.TaosConnection, int64(2<<56|id), true, true, handlerReading.Handler) + if stmt2ReadingPointer == nil { + panic(fmt.Errorf("failed to create stmt2")) + } + code := wrapper.TaosStmt2Prepare(stmt2ReadingPointer, prepareReadingSql) + if code != 0 { + errMsg := wrapper.TaosStmt2Error(stmt2ReadingPointer) + panic(fmt.Errorf("failed to prepare stmt2: %s", errMsg)) + } + + handlerDiagnostics := async.GlobalAsync.HandlerPool.Get() + prepareDiagnosticsSql := "insert into ? values(?,?,?,?)" + stmt2DiagnosticsPointer := wrapper.TaosStmt2Init(p._db.TaosConnection, int64(3<<56|id), true, true, handlerDiagnostics.Handler) + if stmt2DiagnosticsPointer == nil { + panic(fmt.Errorf("failed to create stmt2")) + } + code = wrapper.TaosStmt2Prepare(stmt2DiagnosticsPointer, prepareDiagnosticsSql) + if code != 0 { + errMsg := wrapper.TaosStmt2Error(stmt2DiagnosticsPointer) + panic(fmt.Errorf("failed to prepare stmt2: %s", errMsg)) + } + + // init table name c buffer + readingsTableIndex := p.partitionTable[SuperTableReadings][id] + p.tableNamePointerCache[SuperTableReadings] = make([]unsafe.Pointer, len(readingsTableIndex)) + p.readingsTableNameCBuffer = C.calloc(C.size_t(24*len(readingsTableIndex)), 1) + cursor := p.readingsTableNameCBuffer + prefix := []byte("r_truck_") + var tableNameBytes []byte + for i := uint64(0); i < uint64(len(readingsTableIndex)); i++ { + if readingsTableIndex[i] == 0 { + tableNameNull := []byte("r_truck_null") + C.memcpy(cursor, unsafe.Pointer(&tableNameNull[0]), C.size_t(len(tableNameNull))) + } else { + tableNameBytes = strconv.AppendUint(prefix, uint64(readingsTableIndex[i]-1), 10) + C.memcpy(cursor, unsafe.Pointer(&tableNameBytes[0]), C.size_t(len(tableNameBytes))) + } + p.tableNamePointerCache[SuperTableReadings][i] = cursor + cursor = unsafe.Pointer(uintptr(cursor) + 24) + } + + diagnosticsTableIndex := p.partitionTable[SuperTableDiagnostics][id] + p.tableNamePointerCache[SuperTableDiagnostics] = make([]unsafe.Pointer, len(diagnosticsTableIndex)) + p.diagnosticsTableNameCBuffer = C.calloc(C.size_t(24*len(diagnosticsTableIndex)), 1) + cursor = p.diagnosticsTableNameCBuffer + prefix = []byte("d_truck_") + for i := uint64(0); i < uint64(len(diagnosticsTableIndex)); i++ { + if diagnosticsTableIndex[i] == 0 { + tableNameNull := []byte("d_truck_null") + C.memcpy(cursor, unsafe.Pointer(&tableNameNull[0]), C.size_t(len(tableNameNull))) + } else { + tableNameBytes = strconv.AppendUint(prefix, uint64(diagnosticsTableIndex[i]-1), 10) + C.memcpy(cursor, unsafe.Pointer(&tableNameBytes[0]), C.size_t(len(tableNameBytes))) + } + p.tableNamePointerCache[SuperTableDiagnostics][i] = cursor + cursor = unsafe.Pointer(uintptr(cursor) + 24) + } + + // crate stmt2 bind buffer + cBuffer := allocBuffer(len(readingsTableIndex), int(p.batchSize), []int{ + 8, + 8, + 8, + 8, + 8, + 8, + 8, + 8, + }, []int{ + taosCommon.TSDB_DATA_TYPE_TIMESTAMP, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + }) + p.cBuffers[ReadingsHandleIndex] = cBuffer + p.stmt2CHandle[ReadingsHandleIndex] = stmt2ReadingPointer + p.stmt2CBHandle[ReadingsHandleIndex] = handlerReading + + cBuffer = allocBuffer(len(diagnosticsTableIndex), int(p.batchSize), []int{ + 8, + 8, + 8, + 8, + }, []int{ + taosCommon.TSDB_DATA_TYPE_TIMESTAMP, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_DOUBLE, + taosCommon.TSDB_DATA_TYPE_BIGINT, + }) + p.cBuffers[DiagnosticsHandleIndex] = cBuffer + p.stmt2CHandle[DiagnosticsHandleIndex] = stmt2DiagnosticsPointer + p.stmt2CBHandle[DiagnosticsHandleIndex] = handlerDiagnostics + + // init slot + p.readingsSlot = make([][][]byte, len(readingsTableIndex)) + for i := 0; i < len(readingsTableIndex); i++ { + p.readingsSlot[i] = make([][]byte, 0, 10) + } + p.readingsBatchIndexer = make([]uint32, 0, p.batchSize) + + p.diagnosticsSlot = make([][][]byte, len(diagnosticsTableIndex)) + for i := 0; i < len(diagnosticsTableIndex); i++ { + p.diagnosticsSlot[i] = make([][]byte, 0, 10) + } + p.diagnosticsBatchIndexer = make([]uint32, 0, p.batchSize) + + p.receiveReadingsExecResult() + p.receiveDiagnosticsExecResult() + } +} + +func (p *processor) receiveHostExecResult() { + p.finishWg.Add(1) + go func() { + defer p.finishWg.Done() + stmtHandler := p.stmt2CHandle[CpuHandleIndex] + handle := p.stmt2CBHandle[CpuHandleIndex] + exitSign := p.exitSign + for { + select { + case result := <-handle.Caller.ExecResult: + if result.Code != 0 { + errStr := wrapper.TaosStmt2Error(stmtHandler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", result.Code, errStr)) + } + atomic.AddInt32(&p.hostExecCount, -1) + case <-exitSign: + c := atomic.LoadInt32(&p.hostExecCount) + if c == 0 { + return + } + for i := int32(0); i < c; i++ { + result := <-handle.Caller.ExecResult + if result.Code != 0 { + errStr := wrapper.TaosStmt2Error(stmtHandler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", result.Code, errStr)) + } + } + return + } + } + }() +} + +func (p *processor) receiveReadingsExecResult() { + p.finishWg.Add(1) + go func() { + defer p.finishWg.Done() + stmtHandler := p.stmt2CHandle[ReadingsHandleIndex] + handle := p.stmt2CBHandle[ReadingsHandleIndex] + exitSign := p.exitSign + for { + select { + case result := <-handle.Caller.ExecResult: + if result.Code != 0 { + errStr := wrapper.TaosStmt2Error(stmtHandler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", result.Code, errStr)) + } + atomic.AddInt32(&p.readingsExecCount, -1) + case <-exitSign: + c := atomic.LoadInt32(&p.readingsExecCount) + if c == 0 { + return + } + for i := int32(0); i < c; i++ { + result := <-handle.Caller.ExecResult + if result.Code != 0 { + errStr := wrapper.TaosStmt2Error(stmtHandler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", result.Code, errStr)) + } + } + return + } + } + }() +} + +func (p *processor) receiveDiagnosticsExecResult() { + p.finishWg.Add(1) + go func() { + defer p.finishWg.Done() + stmtHandler := p.stmt2CHandle[DiagnosticsHandleIndex] + handle := p.stmt2CBHandle[DiagnosticsHandleIndex] + exitSign := p.exitSign + for { + select { + case result := <-handle.Caller.ExecResult: + if result.Code != 0 { + errStr := wrapper.TaosStmt2Error(stmtHandler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", result.Code, errStr)) + } + atomic.AddInt32(&p.diagnosticsExecCount, -1) + case <-exitSign: + c := atomic.LoadInt32(&p.diagnosticsExecCount) + if c == 0 { + return + } + for i := int32(0); i < c; i++ { + result := <-handle.Caller.ExecResult + if result.Code != 0 { + errStr := wrapper.TaosStmt2Error(stmtHandler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", result.Code, errStr)) + } + } + return + } + } + }() +} + +//typedef struct TAOS_STMT2_BINDV { +// int count; +// char **tbnames; +// TAOS_STMT2_BIND **tags; +// TAOS_STMT2_BIND **bind_cols; +//} TAOS_STMT2_BINDV; + +// typedef struct TAOS_STMT2_BIND { +// int buffer_type; +// void *buffer; +// int32_t *length; +// char *is_null; +// int num; +//} TAOS_STMT2_BIND; + +const BindVSize = uintptr(C.sizeof_struct_TAOS_STMT2_BINDV) +const BindSize = uintptr(C.sizeof_struct_TAOS_STMT2_BIND) + +func allocBuffer(partitionTables int, batchSize int, colSize []int, colTypes []int) *bufferPointer { + colCount := len(colSize) + bufferSize := 0 + for i := 0; i < colCount; i++ { + bufferSize += colSize[i] + } + bindTables := partitionTables + if bindTables > batchSize { + bindTables = batchSize + } + maxRow := batchSize + /* + + | TAOS_STMT2_BINDV | + | tbnames pointer 1 | tbnames pointer 2 | tbnames pointer 3 | tbnames pointer ... | maxTable + | bind_cols pointer 1 | bind_cols pointer 2 | bind_cols pointer 3 | bind_cols pointer ... | maxTable * colCount + | TAOS_STMT2_BIND 1 | TAOS_STMT2_BIND 2 | TAOS_STMT2_BIND 3 | TAOS_STMT2_BIND ... | maxTable * colCount + */ + + // bindv + tbname pointer + bindcols pointer + bind struct pointer + bind structs + fixedSize := + // TAOS_STMT2_BINDV + BindVSize + + // table name pointer + wrapper.PointerSize*uintptr(bindTables) + + // bind_cols pointer + wrapper.PointerSize*uintptr(bindTables*colCount) + + // bind_cols TAOS_STMT2_BIND + BindSize*uintptr(bindTables*colCount) + + dataSize := uintptr(bufferSize*maxRow) + 1 + isNullSize := uintptr(maxRow*colCount) + 1 + totalSize := fixedSize + dataSize + isNullSize + _ = totalSize + bindVPointer := unsafe.Pointer(C.calloc(C.size_t(fixedSize), 1)) + dataPointer := unsafe.Pointer(C.calloc(C.size_t(dataSize), 1)) + isNullPointer := unsafe.Pointer(C.calloc(C.size_t(isNullSize), 1)) + + // bindV + bindV := (*C.TAOS_STMT2_BINDV)(bindVPointer) + // tableName + tableNamesPointer := unsafe.Pointer(uintptr(bindVPointer) + BindVSize) + bindV.tbnames = (**C.char)(tableNamesPointer) + + // bind_cols pointer + bindStructsPointerPointer := unsafe.Pointer(uintptr(tableNamesPointer) + uintptr(bindTables)*wrapper.PointerSize) + bindV.bind_cols = (**C.TAOS_STMT2_BIND)(bindStructsPointerPointer) + + // TAOS_STMT2_BIND + bindStructsPointer := unsafe.Pointer(uintptr(bindStructsPointerPointer) + uintptr(bindTables*colCount)*wrapper.PointerSize) + + tableNamesPointers := make([]unsafe.Pointer, bindTables) + bindPointers := make([][]unsafe.Pointer, bindTables) + var bind *C.TAOS_STMT2_BIND + for tableIndex := 0; tableIndex < bindTables; tableIndex++ { + // table name pointer + tableNamesPointers[tableIndex] = tableNamesPointer + tableNamesPointer = unsafe.Pointer(uintptr(tableNamesPointer) + wrapper.PointerSize) + bindPointers[tableIndex] = make([]unsafe.Pointer, colCount) + // set bind struct pointer + *(**C.TAOS_STMT2_BIND)(unsafe.Pointer(uintptr(bindStructsPointerPointer) + uintptr(tableIndex)*wrapper.PointerSize)) = (*C.TAOS_STMT2_BIND)(bindStructsPointer) + + for colIndex := 0; colIndex < colCount; colIndex++ { + // bind struct pointer + bindPointers[tableIndex][colIndex] = bindStructsPointer + // set col type + bind = (*C.TAOS_STMT2_BIND)(bindStructsPointer) + bind.buffer_type = (C.int)(colTypes[colIndex]) + bindStructsPointer = unsafe.Pointer(uintptr(bindStructsPointer) + BindSize) + } + } + + tableColDataPointers := make([]unsafe.Pointer, colCount) + isNullPointers := make([]unsafe.Pointer, colCount) + for colIndex := 0; colIndex < colCount; colIndex++ { + tableColDataPointers[colIndex] = dataPointer + dataPointer = unsafe.Pointer(uintptr(dataPointer) + uintptr(colSize[colIndex]*maxRow)) + isNullPointers[colIndex] = isNullPointer + isNullPointer = unsafe.Pointer(uintptr(isNullPointer) + uintptr(maxRow)) + } + + buffer := &bufferPointer{ + bindVP: bindVPointer, + tableNameP: tableNamesPointers, + bindsP: bindPointers, + colP: tableColDataPointers, + isNullP: isNullPointers, + } + return buffer +} + +func (p *processor) ProcessBatch(b targets.Batch, doLoad bool) (metricCount, rowCount uint64) { + batches := b.(*hypertableArr) + metricCnt := batches.totalMetric + if !doLoad { + return metricCnt, uint64(batches.cnt) + } + rowCount = uint64(batches.cnt) + if len(batches.createSql) > 0 { + p.createWg.Add(1) + go func() { + sqls := batches.createSql + p.cBufPosition = unsafe.Pointer(uintptr(p.cBuf) + 12) + p.bufIndex = 12 + for i := range sqls { + sh := (*reflect.SliceHeader)(unsafe.Pointer(sqls[i])) + createSize := sh.Len - 6 + if p.bufIndex+createSize >= Size1M { + *(*C.char)(p.cBufPosition) = C.char(0) + C.taos_query_a_wrapper2(p._db.TaosConnection, (*C.char)(p.cBuf), p.hp) + res := <-p.caller.QueryResult + if res.N != 0 { + panic(wrapper.TaosErrorStr(res.Res)) + } + go func(result unsafe.Pointer) { + C.taos_free_result(result) + }(res.Res) + p.cBufPosition = unsafe.Pointer(uintptr(p.cBuf) + 12) + p.bufIndex = 12 + } + C.memcpy(p.cBufPosition, unsafe.Pointer(sh.Data+6), C.size_t(createSize)) + p.cBufPosition = unsafe.Pointer(uintptr(p.cBufPosition) + uintptr(createSize)) + p.bufIndex += createSize + } + if p.bufIndex > 12 { + *(*C.char)(p.cBufPosition) = C.char(0) + C.taos_query_a_wrapper2(p._db.TaosConnection, (*C.char)(p.cBuf), p.hp) + res := <-p.caller.QueryResult + if res.N != 0 { + panic(wrapper.TaosErrorStr(res.Res)) + } + go func(result unsafe.Pointer) { + C.taos_free_result(result) + }(res.Res) + } + p.createWg.Done() + }() + } + + switch p.useCase { + case CpuCase: + if len(batches.data) > 0 { + var bind *C.TAOS_STMT2_BIND + hostTableIndex := 0 + hostCBuffers := p.cBuffers[CpuHandleIndex] + var nullByte byte + var currentRowData []byte + var dataPointer unsafe.Pointer + currentIsNullPointer := [11]unsafe.Pointer{ + hostCBuffers.isNullP[0], + hostCBuffers.isNullP[1], + hostCBuffers.isNullP[2], + hostCBuffers.isNullP[3], + hostCBuffers.isNullP[4], + hostCBuffers.isNullP[5], + hostCBuffers.isNullP[6], + hostCBuffers.isNullP[7], + hostCBuffers.isNullP[8], + hostCBuffers.isNullP[9], + hostCBuffers.isNullP[10], + } + currentDataPointer := [11]unsafe.Pointer{ + hostCBuffers.colP[0], + hostCBuffers.colP[1], + hostCBuffers.colP[2], + hostCBuffers.colP[3], + hostCBuffers.colP[4], + hostCBuffers.colP[5], + hostCBuffers.colP[6], + hostCBuffers.colP[7], + hostCBuffers.colP[8], + hostCBuffers.colP[9], + hostCBuffers.colP[10], + } + for i := 0; i < len(batches.data); i++ { + row := *(batches.data[i]) + slotID := p.tableSlot[SuperTableHost][*(*uint32)(unsafe.Pointer(&row[2]))] + p.hostBatchIndexer = append(p.hostBatchIndexer, slotID) + p.hostSlot[slotID] = append(p.hostSlot[slotID], row) + } + for _, slotID := range p.hostBatchIndexer { + rowData := p.hostSlot[slotID] + if len(rowData) == 0 { + continue + } + p.hostSlot[slotID] = p.hostSlot[slotID][:0] + rowLen := len(rowData) + for i := 0; i < len(rowData); i++ { + tmp := rowData[i] + currentRowData = tmp[7:len(tmp):len(tmp)] + nullByte = currentRowData[0] + dataPointer = unsafe.Pointer(¤tRowData[2]) + + *(*C.int64_t)(currentDataPointer[0]) = *(*C.int64_t)(dataPointer) + + currentDataPointer[0] = unsafe.Pointer(uintptr(currentDataPointer[0]) + 8) + dataPointer = unsafe.Pointer(uintptr(dataPointer) + 8) + + // set col data and is_null + for colIndex := 1; colIndex < 11; colIndex++ { + if colIndex == 8 { + nullByte = currentRowData[1] + } + isNullP := currentIsNullPointer[colIndex] + dataP := currentDataPointer[colIndex] + if nullByte&(1<<(7-(colIndex&7))) != 0 { + *(*C.char)(currentIsNullPointer[colIndex]) = C.char(0) + } else { + *(*C.char)(isNullP) = C.char(0) + *(*C.int64_t)(dataP) = *(*C.int64_t)(dataPointer) + } + + currentIsNullPointer[colIndex] = unsafe.Pointer(uintptr(isNullP) + 1) + currentDataPointer[colIndex] = unsafe.Pointer(uintptr(dataP) + 8) + dataPointer = unsafe.Pointer(uintptr(dataPointer) + 8) + } + + // first row, set buffer and is_null pointer + if i == 0 { + for colIndex := 0; colIndex < 11; colIndex++ { + bind = (*C.TAOS_STMT2_BIND)(hostCBuffers.bindsP[hostTableIndex][colIndex]) + bind.buffer = unsafe.Pointer(uintptr(currentDataPointer[colIndex]) - 8) + bind.is_null = (*C.char)(unsafe.Pointer(uintptr(currentIsNullPointer[colIndex]) - 1)) + bind.num = (C.int)(rowLen) + } + } + } + *(**C.char)(hostCBuffers.tableNameP[hostTableIndex]) = (*C.char)(p.tableNamePointerCache[SuperTableHost][slotID]) + hostTableIndex += 1 + } + p.hostBatchIndexer = p.hostBatchIndexer[:0] + + // stmt2 bind + bindv := (*C.TAOS_STMT2_BINDV)(hostCBuffers.bindVP) + bindv.count = C.int(hostTableIndex) + handler := p.stmt2CHandle[CpuHandleIndex] + if len(batches.createSql) > 0 { + p.createWg.Wait() + } + code := int(C.taos_stmt2_bind_param(handler, bindv, C.int32_t(-1))) + if code != 0 { + errStr := wrapper.TaosStmt2Error(handler) + panic(fmt.Errorf("failed to bind param stmt2: %d:%s", code, errStr)) + } + code = wrapper.TaosStmt2Exec(handler) + if code != 0 { + errStr := wrapper.TaosStmt2Error(handler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", code, errStr)) + } + atomic.AddInt32(&p.hostExecCount, 1) + } + + case IoTCase: + var row []byte + for i := 0; i < len(batches.data); i++ { + row = *batches.data[i] + if row[1] == SuperTableReadings { + slotID := p.tableSlot[SuperTableReadings][*(*uint32)(unsafe.Pointer(&row[2]))] + p.readingsBatchIndexer = append(p.readingsBatchIndexer, slotID) + p.readingsSlot[slotID] = append(p.readingsSlot[slotID], row) + } else { + slotID := p.tableSlot[SuperTableDiagnostics][*(*uint32)(unsafe.Pointer(&row[2]))] + p.diagnosticsBatchIndexer = append(p.diagnosticsBatchIndexer, slotID) + p.diagnosticsSlot[slotID] = append(p.diagnosticsSlot[slotID], row) + } + } + p.wg.Add(2) + go func() { + if len(p.readingsBatchIndexer) > 0 { + var bind *C.TAOS_STMT2_BIND + readingTableIndex := 0 + readingCBuffers := p.cBuffers[ReadingsHandleIndex] + var nullByte byte + var currentRowData []byte + var dataPointer unsafe.Pointer + currentIsNullPointer := [8]unsafe.Pointer{ + readingCBuffers.isNullP[0], + readingCBuffers.isNullP[1], + readingCBuffers.isNullP[2], + readingCBuffers.isNullP[3], + readingCBuffers.isNullP[4], + readingCBuffers.isNullP[5], + readingCBuffers.isNullP[6], + readingCBuffers.isNullP[7], + } + currentDataPointer := [8]unsafe.Pointer{ + readingCBuffers.colP[0], + readingCBuffers.colP[1], + readingCBuffers.colP[2], + readingCBuffers.colP[3], + readingCBuffers.colP[4], + readingCBuffers.colP[5], + readingCBuffers.colP[6], + readingCBuffers.colP[7], + } + + for _, slotID := range p.readingsBatchIndexer { + rowData := p.readingsSlot[slotID] + if len(rowData) == 0 { + continue + } + p.readingsSlot[slotID] = p.readingsSlot[slotID][:0] + rowLen := len(rowData) + for i := 0; i < len(rowData); i++ { + tmp := rowData[i] + currentRowData = tmp[7:len(tmp):len(tmp)] + nullByte = currentRowData[0] + dataPointer = unsafe.Pointer(¤tRowData[1]) + + *(*C.int64_t)(currentDataPointer[0]) = *(*C.int64_t)(dataPointer) + + currentDataPointer[0] = unsafe.Pointer(uintptr(currentDataPointer[0]) + 8) + dataPointer = unsafe.Pointer(uintptr(dataPointer) + 8) + + for colIndex := 1; colIndex < 8; colIndex++ { + if nullByte&(1<<(7-colIndex)) != 0 { + *(*C.char)(currentIsNullPointer[colIndex]) = C.char(1) + } else { + *(*C.char)(currentIsNullPointer[colIndex]) = C.char(0) + *(*C.int64_t)(currentDataPointer[colIndex]) = *(*C.int64_t)(dataPointer) + } + + currentIsNullPointer[colIndex] = unsafe.Pointer(uintptr(currentIsNullPointer[colIndex]) + 1) + currentDataPointer[colIndex] = unsafe.Pointer(uintptr(currentDataPointer[colIndex]) + 8) + dataPointer = unsafe.Pointer(uintptr(dataPointer) + 8) + } + + if i == 0 { + // first row set buffer and is_null pointer + for colIndex := 0; colIndex < 8; colIndex++ { + bind = (*C.TAOS_STMT2_BIND)(readingCBuffers.bindsP[readingTableIndex][colIndex]) + bind.buffer = unsafe.Pointer(uintptr(currentDataPointer[colIndex]) - 8) + bind.is_null = (*C.char)(unsafe.Pointer(uintptr(currentIsNullPointer[colIndex]) - 1)) + bind.num = (C.int)(rowLen) + } + } + } + *(**C.char)(readingCBuffers.tableNameP[readingTableIndex]) = (*C.char)(p.tableNamePointerCache[SuperTableReadings][slotID]) + readingTableIndex += 1 + } + p.readingsBatchIndexer = p.readingsBatchIndexer[:0] + bindv := (*C.TAOS_STMT2_BINDV)(readingCBuffers.bindVP) + bindv.count = C.int(readingTableIndex) + handler := p.stmt2CHandle[ReadingsHandleIndex] + + if len(batches.createSql) > 0 { + p.createWg.Wait() + } + code := int(C.taos_stmt2_bind_param(handler, bindv, C.int32_t(-1))) + if code != 0 { + errStr := wrapper.TaosStmt2Error(handler) + panic(fmt.Errorf("failed to bind param stmt2: %d:%s", code, errStr)) + } + code = wrapper.TaosStmt2Exec(handler) + if code != 0 { + errStr := wrapper.TaosStmt2Error(handler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", code, errStr)) + } + atomic.AddInt32(&p.readingsExecCount, 1) + } + p.wg.Done() + }() + go func() { + if len(p.diagnosticsBatchIndexer) > 0 { + var bind *C.TAOS_STMT2_BIND + diagnosticsTableIndex := 0 + diagnosticsCBuffers := p.cBuffers[DiagnosticsHandleIndex] + var nullByte byte + var currentRowData []byte + var dataPointer unsafe.Pointer + currentIsNullPointer := [4]unsafe.Pointer{ + diagnosticsCBuffers.isNullP[0], + diagnosticsCBuffers.isNullP[1], + diagnosticsCBuffers.isNullP[2], + diagnosticsCBuffers.isNullP[3], + } + currentDataPointer := [4]unsafe.Pointer{ + diagnosticsCBuffers.colP[0], + diagnosticsCBuffers.colP[1], + diagnosticsCBuffers.colP[2], + diagnosticsCBuffers.colP[3], + } + + for _, slotID := range p.diagnosticsBatchIndexer { + rowData := p.diagnosticsSlot[slotID] + if len(rowData) == 0 { + continue + } + p.diagnosticsSlot[slotID] = p.diagnosticsSlot[slotID][:0] + rowLen := len(rowData) + for i := 0; i < len(rowData); i++ { + tmp := rowData[i] + currentRowData = tmp[7:len(tmp):len(tmp)] + nullByte = currentRowData[0] + dataPointer = unsafe.Pointer(¤tRowData[1]) + + *(*C.int64_t)(currentDataPointer[0]) = *(*C.int64_t)(dataPointer) + + currentDataPointer[0] = unsafe.Pointer(uintptr(currentDataPointer[0]) + 8) + dataPointer = unsafe.Pointer(uintptr(dataPointer) + 8) + + // col1 + if nullByte&(1<<6) != 0 { + *(*C.char)(currentIsNullPointer[1]) = C.char(1) + } else { + *(*C.char)(currentIsNullPointer[1]) = C.char(0) + *(*C.double)(currentDataPointer[1]) = *(*C.double)(dataPointer) + } + currentIsNullPointer[1] = unsafe.Pointer(uintptr(currentIsNullPointer[1]) + 1) + currentDataPointer[1] = unsafe.Pointer(uintptr(currentDataPointer[1]) + 8) + dataPointer = unsafe.Pointer(uintptr(dataPointer) + 8) + + // col2 + if nullByte&(1<<5) != 0 { + *(*C.char)(currentIsNullPointer[2]) = C.char(1) + } else { + *(*C.char)(currentIsNullPointer[2]) = C.char(0) + *(*C.double)(currentDataPointer[2]) = *(*C.double)(dataPointer) + } + currentIsNullPointer[2] = unsafe.Pointer(uintptr(currentIsNullPointer[2]) + 1) + currentDataPointer[2] = unsafe.Pointer(uintptr(currentDataPointer[2]) + 8) + dataPointer = unsafe.Pointer(uintptr(dataPointer) + 8) + + // col3 + if nullByte&(1<<4) != 0 { + *(*C.char)(currentIsNullPointer[3]) = C.char(1) + } else { + *(*C.char)(currentIsNullPointer[3]) = C.char(0) + *(*C.int64_t)(currentDataPointer[3]) = *(*C.int64_t)(dataPointer) + } + currentIsNullPointer[3] = unsafe.Pointer(uintptr(currentIsNullPointer[3]) + 1) + currentDataPointer[3] = unsafe.Pointer(uintptr(currentDataPointer[3]) + 8) + if i == 0 { + // first row set buffer and is_null pointer + for colIndex := 0; colIndex < 4; colIndex++ { + bind = (*C.TAOS_STMT2_BIND)(diagnosticsCBuffers.bindsP[diagnosticsTableIndex][colIndex]) + bind.buffer = unsafe.Pointer(uintptr(currentDataPointer[colIndex]) - 8) + bind.is_null = (*C.char)(unsafe.Pointer(uintptr(currentIsNullPointer[colIndex]) - 1)) + bind.num = (C.int)(rowLen) + } + } + } + *(**C.char)(diagnosticsCBuffers.tableNameP[diagnosticsTableIndex]) = (*C.char)(p.tableNamePointerCache[SuperTableDiagnostics][slotID]) + diagnosticsTableIndex += 1 + } + p.diagnosticsBatchIndexer = p.diagnosticsBatchIndexer[:0] + bindv := (*C.TAOS_STMT2_BINDV)(diagnosticsCBuffers.bindVP) + bindv.count = C.int(diagnosticsTableIndex) + handler := p.stmt2CHandle[DiagnosticsHandleIndex] + if len(batches.createSql) > 0 { + p.createWg.Wait() + } + code := int(C.taos_stmt2_bind_param(handler, bindv, C.int32_t(-1))) + if code != 0 { + errStr := wrapper.TaosStmt2Error(handler) + panic(fmt.Errorf("failed to bind param stmt2: %d:%s", code, errStr)) + } + code = wrapper.TaosStmt2Exec(handler) + if code != 0 { + errStr := wrapper.TaosStmt2Error(handler) + panic(fmt.Errorf("failed to exec stmt2: %d:%s", code, errStr)) + } + atomic.AddInt32(&p.diagnosticsExecCount, 1) + } + + p.wg.Done() + }() + p.wg.Wait() + } + go func() { + globalSlicePool.Put(batches.data) + }() + return metricCnt, rowCount +} + +func (p *processor) Close(doLoad bool) { + if doLoad { + close(p.exitSign) + p.finishWg.Wait() + } +} diff --git a/pkg/targets/tdenginestmt2/program_options.go b/pkg/targets/tdenginestmt2/program_options.go new file mode 100644 index 000000000..cd8664c2f --- /dev/null +++ b/pkg/targets/tdenginestmt2/program_options.go @@ -0,0 +1,5 @@ +package tdenginestmt2 + +import "github.com/timescale/tsbs/pkg/targets/tdengine" + +type LoadingOptions tdengine.LoadingOptions diff --git a/pkg/targets/tdenginestmt2/scan.go b/pkg/targets/tdenginestmt2/scan.go new file mode 100644 index 000000000..635905214 --- /dev/null +++ b/pkg/targets/tdenginestmt2/scan.go @@ -0,0 +1,207 @@ +package tdenginestmt2 + +import ( + "bytes" + "fmt" + "unsafe" + + "github.com/spaolacci/murmur3" + "github.com/timescale/tsbs/pkg/data" + "github.com/timescale/tsbs/pkg/targets" +) + +// Indexer is used to consistently send the same hostnames to the same worker +type Indexer struct { + cache [3][]uint +} + +func NewIndexer(prefix []byte, partitions int, hashEndGroups []uint32, useCase byte, scale uint32) (_ *Indexer, _ [3][]uint32, hostTableIndex [][]uint32, readingsTableIndex [][]uint32, diagnosticsTableIndex [][]uint32) { + cache := [3][]uint{} + buf := &bytes.Buffer{} + var idx uint32 + switch useCase { + case CpuCase: + cache[SuperTableHost] = make([]uint, scale+1) + hostTableIndex = make([][]uint32, partitions) + //partitionIndex := make([]uint32, partitions) + tableOffset := make([]uint32, scale+1) + buf.Write(prefix) + buf.WriteString("host_null") + hash := murmur3.Sum32WithSeed(buf.Bytes(), 0x12345678) + buf.Reset() + for j := 0; j < partitions; j++ { + if hash <= hashEndGroups[j] { + cache[SuperTableHost][0] = uint(j) + tableOffset[0] = uint32(len(hostTableIndex[j])) + hostTableIndex[j] = append(hostTableIndex[j], 0) + break + } + } + tbPrefix := append(prefix, []byte("host_")...) + for i := uint32(0); i < scale; i++ { + buf.Write(tbPrefix) + _, err := fmt.Fprintf(buf, "%d", i) + if err != nil { + panic(err) + } + hash = murmur3.Sum32WithSeed(buf.Bytes(), 0x12345678) + buf.Reset() + for j := 0; j < partitions; j++ { + if hash <= hashEndGroups[j] { + idx = i + 1 + cache[SuperTableHost][idx] = uint(j) + tableOffset[idx] = uint32(len(hostTableIndex[j])) + hostTableIndex[j] = append(hostTableIndex[j], idx) + break + } + } + } + return &Indexer{ + cache: cache, + }, + [3][]uint32{ + tableOffset, + }, + hostTableIndex, + nil, + nil + case IoTCase: + cache[SuperTableReadings] = make([]uint, scale+1) + readingsTableIndex = make([][]uint32, partitions) + rTableOffset := make([]uint32, scale+1) + buf.Write(prefix) + buf.WriteString("r_truck_null") + hash := murmur3.Sum32WithSeed(buf.Bytes(), 0x12345678) + buf.Reset() + for j := 0; j < partitions; j++ { + if hash <= hashEndGroups[j] { + cache[SuperTableReadings][0] = uint(j) + rTableOffset[0] = uint32(len(readingsTableIndex[j])) + readingsTableIndex[j] = append(readingsTableIndex[j], 0) + break + } + } + tbPrefix := append(prefix, []byte("r_truck_")...) + for i := uint32(0); i < scale; i++ { + buf.Write(tbPrefix) + _, err := fmt.Fprintf(buf, "%d", i) + if err != nil { + panic(err) + } + hash := murmur3.Sum32WithSeed(buf.Bytes(), 0x12345678) + buf.Reset() + for j := 0; j < partitions; j++ { + if hash <= hashEndGroups[j] { + idx = i + 1 + cache[SuperTableReadings][idx] = uint(j) + rTableOffset[idx] = uint32(len(readingsTableIndex[j])) + readingsTableIndex[j] = append(readingsTableIndex[j], idx) + break + } + } + } + + cache[SuperTableDiagnostics] = make([]uint, scale+1) + diagnosticsTableIndex = make([][]uint32, partitions) + dTableOffset := make([]uint32, scale+1) + buf.Write(prefix) + buf.WriteString("d_truck_null") + hash = murmur3.Sum32WithSeed(buf.Bytes(), 0x12345678) + buf.Reset() + for j := 0; j < partitions; j++ { + if hash <= hashEndGroups[j] { + cache[SuperTableDiagnostics][0] = uint(j) + dTableOffset[0] = uint32(len(diagnosticsTableIndex[j])) + diagnosticsTableIndex[j] = append(diagnosticsTableIndex[j], 0) + break + } + } + tbPrefix = append(prefix, []byte("d_truck_")...) + for i := uint32(0); i < scale; i++ { + buf.Write(tbPrefix) + _, err := fmt.Fprintf(buf, "%d", i) + if err != nil { + panic(err) + } + hash := murmur3.Sum32WithSeed(buf.Bytes(), 0x12345678) + buf.Reset() + for j := 0; j < partitions; j++ { + if hash <= hashEndGroups[j] { + idx = i + 1 + cache[SuperTableDiagnostics][idx] = uint(j) + dTableOffset[idx] = uint32(len(diagnosticsTableIndex[j])) + diagnosticsTableIndex[j] = append(diagnosticsTableIndex[j], idx) + break + } + } + } + + return &Indexer{ + cache: cache, + }, + [3][]uint32{ + nil, + rTableOffset, + dTableOffset, + }, + nil, + readingsTableIndex, + diagnosticsTableIndex + default: + panic(fmt.Sprintf("invalid use case: %d", useCase)) + } +} + +func (i *Indexer) GetIndex(item data.LoadedPoint) uint { + p := *item.Data.(*[]byte) + return i.cache[p[1]][*(*uint32)(unsafe.Pointer(&p[2]))] +} + +type hypertableArr struct { + data []*[]byte + createSql []*[]byte + totalMetric uint64 + cnt uint +} + +func (ha *hypertableArr) Len() uint { + return ha.cnt +} + +func (ha *hypertableArr) Append(item data.LoadedPoint) { + p := item.Data.(*[]byte) + s := *p + _ = s[7] + if s[0] == InsertData { + if s[6] != 1 { + ha.data = append(ha.data, p) + } + switch s[1] { + case SuperTableHost: + ha.totalMetric += 10 + case SuperTableReadings: + ha.totalMetric += 7 + case SuperTableDiagnostics: + ha.totalMetric += 3 + default: + fatal("invalid table type:%d", s[1]) + } + ha.cnt++ + } else { + ha.createSql = append(ha.createSql, p) + } +} + +type BatchFactory struct { + batchSize uint +} + +func (b *BatchFactory) New() targets.Batch { + return &hypertableArr{ + data: make([]*[]byte, 0, b.batchSize), + } +} + +func NewBatchFactory() targets.BatchFactory { + return &BatchFactory{} +} diff --git a/pkg/targets/tdenginestmt2/serializer.go b/pkg/targets/tdenginestmt2/serializer.go new file mode 100644 index 000000000..a6ad5a6e6 --- /dev/null +++ b/pkg/targets/tdenginestmt2/serializer.go @@ -0,0 +1,558 @@ +package tdenginestmt2 + +import ( + "bytes" + "encoding/binary" + "fmt" + "io" + "log" + "math" + "strconv" + "strings" + "unsafe" + + "github.com/timescale/tsbs/pkg/data" + "github.com/timescale/tsbs/pkg/data/usecases/common" +) + +const ( + SuperTableHost = 0 + SuperTableReadings = 1 + SuperTableDiagnostics = 2 +) + +type Serializer struct { + tmpBuf *bytes.Buffer + writeBuf *bytes.Buffer + useCase byte + scale uint32 + cpuTableList []int64 + readingsTableList []int64 + diagnosticsTableList []int64 + superTable [3]*Table +} + +type Table struct { + sortColumns map[string]int + sortTags map[string]int + tagBase []string + nullBit []byte + colOffset []int + colBufferLength int +} + +func FastFormat(buf *bytes.Buffer, v interface{}) { + switch v := v.(type) { + case int: + buf.WriteString(strconv.Itoa(v)) + case int64: + buf.WriteString(strconv.FormatInt(v, 10)) + case float64: + buf.WriteString(strconv.FormatFloat(v, 'f', -1, 64)) + case float32: + buf.WriteString(strconv.FormatFloat(float64(v), 'f', -1, 32)) + case bool: + buf.WriteString(strconv.FormatBool(v)) + case []byte: + buf.WriteByte('\'') + buf.Write(v) + buf.WriteByte('\'') + case string: + buf.WriteByte('\'') + buf.WriteString(v) + buf.WriteByte('\'') + case nil: + buf.WriteString("null") + default: + panic(fmt.Sprintf("unknown field type for %#v", v)) + } +} + +type tbNameRule struct { + tag string + prefix string + nilValue string +} + +var tbRule = []*tbNameRule{ + { + tag: "hostname", + nilValue: "host_null", + }, + { + tag: "name", + prefix: "r_", + nilValue: "r_truck_null", + }, + { + tag: "name", + prefix: "d_", + nilValue: "d_truck_null", + }, +} + +const ( + CpuCase = 1 + IoTCase = 2 +) + +/* + fixed header + +| version(1 byte) | case (1 byte) | scale (4 bytes) + +*/ + +func (s *Serializer) Config(config *common.DataGeneratorConfig, w io.Writer) error { + if config.Scale > math.MaxUint32-1 { + return fmt.Errorf("scale %d is too large", config.Scale) + } + s.scale = uint32(config.Scale) + useCase := s.prepare(config.BaseConfig.Use) + if useCase == 0 { + return fmt.Errorf("use case %s not supported", config.BaseConfig.Use) + } + bs := make([]byte, 6) + bs[0] = 1 // version + bs[1] = useCase + binary.LittleEndian.PutUint32(bs[2:], s.scale) + _, err := w.Write(bs) + if err != nil { + return err + } + s.useCase = useCase + switch useCase { + case CpuCase: + s.cpuTableList = make([]int64, config.BaseConfig.Scale+1) + case IoTCase: + s.readingsTableList = make([]int64, config.BaseConfig.Scale+1) + s.diagnosticsTableList = make([]int64, config.BaseConfig.Scale+1) + default: + return fmt.Errorf("use case %s not supported", config.BaseConfig.Use) + } + return nil +} + +var cpuTable = &Table{ + sortColumns: map[string]int{ + "usage_user": 1, + "usage_system": 2, + "usage_idle": 3, + "usage_nice": 4, + "usage_iowait": 5, + "usage_irq": 6, + "usage_softirq": 7, + "usage_steal": 8, + "usage_guest": 9, + "usage_guest_nice": 10, + }, + sortTags: map[string]int{ + "hostname": 0, + "region": 1, + "datacenter": 2, + "rack": 3, + "os": 4, + "arch": 5, + "team": 6, + "service": 7, + "service_version": 8, + "service_environment": 9, + }, + tagBase: []string{ + "null", + "null", + "null", + "null", + "null", + "null", + "null", + "null", + "null", + "null", + }, + nullBit: []byte{0b01111111, 0b11100000}, + colOffset: []int{ + 0, + 8, + 16, + 24, + 32, + 40, + 48, + 56, + 64, + 72, + 80, + }, + colBufferLength: 88, +} +var readingsTable = &Table{ + sortColumns: map[string]int{ + "latitude": 1, + "longitude": 2, + "elevation": 3, + "velocity": 4, + "heading": 5, + "grade": 6, + "fuel_consumption": 7, + }, + sortTags: map[string]int{ + "name": 0, + "fleet": 1, + "driver": 2, + "model": 3, + "device_version": 4, + "load_capacity": 5, + "fuel_capacity": 6, + "nominal_fuel_consumption": 7, + }, + tagBase: []string{ + "null", + "null", + "null", + "null", + "null", + "null", + "null", + "null", + }, + nullBit: []byte{0b01111111}, + colOffset: []int{ + 0, + 8, + 16, + 24, + 32, + 40, + 48, + 56, + }, + colBufferLength: 64, +} +var diagnosticsTable = &Table{ + sortColumns: map[string]int{ + "fuel_state": 1, + "current_load": 2, + "status": 3, + }, + sortTags: map[string]int{ + "name": 0, + "fleet": 1, + "driver": 2, + "model": 3, + "device_version": 4, + "load_capacity": 5, + "fuel_capacity": 6, + "nominal_fuel_consumption": 7, + }, + tagBase: []string{ + "null", + "null", + "null", + "null", + "null", + "null", + "null", + "null", + }, + nullBit: []byte{0b01110000}, + colOffset: []int{ + 0, + 8, + 16, + 24, + }, + colBufferLength: 32, +} + +func (s *Serializer) prepare(use string) uint8 { + switch use { + case common.UseCaseCPUOnly, common.UseCaseCPUSingle: + s.superTable[SuperTableHost] = cpuTable + return CpuCase + case common.UseCaseIoT: + s.superTable[SuperTableReadings] = readingsTable + s.superTable[SuperTableDiagnostics] = diagnosticsTable + return IoTCase + default: + return 0 + } +} + +const ( + CreateTable = 1 + InsertData = 2 +) + +/* + + create table sql + | length (1 or 2 byte) + | type (1 byte,1) | table type(1 byte) | table index (uint32 4 bytes) | sql buffer | +*/ + +/* + insert data + | length (1 byte) + | type (1 byte,2) | table type(1 byte) | table index (uint32 4 bytes) + | duplicate (bool 1 byte)| + | is null bit | column data| +*/ + +func (s *Serializer) Serialize(p *data.Point, w io.Writer) error { + tmpBuf := s.tmpBuf + tagRawKeys := p.TagKeys() + tagRawValues := p.TagValues() + colRawKeys := p.FieldKeys() + colRawValues := p.FieldValues() + superTable := p.MeasurementName() + + var tableList []int64 + superTableType := byte(0) + switch superTable[0] { + case 'c': + superTableType = SuperTableHost + tableList = s.cpuTableList + case 'r': + superTableType = SuperTableReadings + tableList = s.readingsTableList + case 'd': + superTableType = SuperTableDiagnostics + tableList = s.diagnosticsTableList + default: + log.Fatalf("super table: %s not found", superTable) + } + + // generate sub table name + rule := tbRule[superTableType] + fixedName := "" + for index, value := range tagRawValues { + if len(fixedName) == 0 && BytesToString(tagRawKeys[index]) == rule.tag { + str, is := value.(string) + if is { + fixedName = str + } + } + } + + subTable := "" + tbNameIndex := uint32(0) + + if len(fixedName) != 0 { + if len(rule.prefix) == 0 { + subTable = fixedName + } else { + tmpBuf.WriteString(rule.prefix) + tmpBuf.WriteString(fixedName) + subTable = tmpBuf.String() + tmpBuf.Reset() + } + sl := strings.Split(fixedName, "_") + id, err := strconv.ParseInt(sl[len(sl)-1], 10, 32) + if err != nil { + log.Fatalf("table: %s id parse failed", subTable) + } + if id < 0 { + log.Fatalf("table: %s id is negative", subTable) + } + if id > int64(s.scale) { + log.Fatalf("table: %s id is too large", subTable) + } + tbNameIndex = uint32(id + 1) + } else { + // use nil table name + subTable = rule.nilValue + tbNameIndex = 0 + } + + stable := s.superTable[superTableType] + + // check sub table exist + lastTs := tableList[tbNameIndex] + if lastTs == 0 { + var tagValues = make([]string, len(stable.sortTags)) + copy(tagValues, stable.tagBase) + for index, key := range tagRawKeys { + tagIndex, exist := stable.sortTags[BytesToString(key)] + if !exist { + log.Fatalf("stable: %s tag: %s not found", superTable, key) + } + tmpBuf.Reset() + FastFormat(tmpBuf, tagRawValues[index]) + tagValues[tagIndex] = tmpBuf.String() + } + tmpBuf.Reset() + for index, value := range tagValues { + tmpBuf.WriteString(value) + if index != len(tagValues)-1 { + tmpBuf.WriteByte(',') + } + } + //|length | type (1 byte,1) | table type(1 byte) | table index (uint32 4 bytes) | sql buffer | + + sqlBuf := fmt.Sprintf(" %s using %s tags (%s)", subTable, superTable, tmpBuf.Bytes()) + s.writeBuf.Reset() + tmpBuf.Reset() + length := 6 + len(sqlBuf) + + if length < 128 { + s.writeBuf.WriteByte(byte(length)) + } else { + s.writeBuf.WriteByte(byte(length&0x7f | 0x80)) + s.writeBuf.WriteByte(byte(length >> 7)) + } + + // type + s.writeBuf.WriteByte(CreateTable) + // table type + s.writeBuf.WriteByte(superTableType) + // table index + bs := make([]byte, 4) + binary.LittleEndian.PutUint32(bs, tbNameIndex) + s.writeBuf.Write(bs) + // sql length + s.writeBuf.WriteString(sqlBuf) + _, err := w.Write(s.writeBuf.Bytes()) + if err != nil { + return err + } + } + nullBit := make([]byte, len(stable.nullBit)) + copy(nullBit, stable.nullBit) + colBuffer := make([]byte, stable.colBufferLength) + ts := p.TimestampInUnixMs() + duplicate := false + tableList[tbNameIndex] = ts + if ts <= lastTs { + duplicate = true + } + binary.LittleEndian.PutUint64(colBuffer, uint64(ts)) + for index, key := range colRawKeys { + colIndex, exist := stable.sortColumns[BytesToString(key)] + if !exist { + log.Fatalf("stable: %s column: %s not found", superTable, key) + } + value := colRawValues[index] + if value == nil { + continue + } + switch v := value.(type) { + case int: + binary.LittleEndian.PutUint64(colBuffer[stable.colOffset[colIndex]:], uint64(v)) + case int64: + binary.LittleEndian.PutUint64(colBuffer[stable.colOffset[colIndex]:], uint64(v)) + case float64: + binary.LittleEndian.PutUint64(colBuffer[stable.colOffset[colIndex]:], math.Float64bits(v)) + case float32: + binary.LittleEndian.PutUint64(colBuffer[stable.colOffset[colIndex]:], math.Float64bits(float64(v))) + default: + log.Fatalf("stable: %s column: %s type: %T not supported", superTable, key, v) + } + pos := CharOffset(colIndex) + nullBit[pos] = BMUnSetNull(nullBit[pos], colIndex) + } + //insert data + //| length (1 byte) + //| type (1 byte,2) | table type(1 byte) | table index (uint32 4 bytes) + //| duplicate (bool 1 byte)| + //| is null bit | column data| + s.writeBuf.Reset() + length := 7 + len(nullBit) + len(colBuffer) + if length >= 128 { + log.Fatalf("length %d is too large", length) + } + s.writeBuf.WriteByte(byte(length)) + // type + s.writeBuf.WriteByte(InsertData) + // table type + s.writeBuf.WriteByte(superTableType) + // table index + bs := make([]byte, 4) + binary.LittleEndian.PutUint32(bs, tbNameIndex) + s.writeBuf.Write(bs) + // duplicate + if duplicate { + s.writeBuf.WriteByte(1) + } else { + s.writeBuf.WriteByte(0) + } + // is null bit + s.writeBuf.Write(nullBit) + // column data + s.writeBuf.Write(colBuffer) + _, err := w.Write(s.writeBuf.Bytes()) + return err +} + +func BytesToString(b []byte) string { + return *(*string)(unsafe.Pointer(&b)) +} + +func BitPos(n int) int { + return n & (7) +} + +func CharOffset(n int) int { + return n >> 3 +} + +func BMUnSetNull(c byte, n int) byte { + return c - (1 << (7 - BitPos(n))) +} + +const CpuSql = "create stable cpu (" + + "ts timestamp," + + "usage_user bigint," + + "usage_system bigint," + + "usage_idle bigint," + + "usage_nice bigint," + + "usage_iowait bigint," + + "usage_irq bigint," + + "usage_softirq bigint," + + "usage_steal bigint," + + "usage_guest bigint," + + "usage_guest_nice bigint) " + + "tags (" + + "hostname varchar(30)," + + "region varchar(30)," + + "datacenter varchar(30)," + + "rack varchar(30)," + + "os varchar(30)," + + "arch varchar(30)," + + "team varchar(30)," + + "service varchar(30)," + + "service_version varchar(30)," + + "service_environment varchar(30))" +const ReadingsSql = "CREATE STABLE `readings` (" + + "`ts` TIMESTAMP, " + + "`latitude` DOUBLE, " + + "`longitude` DOUBLE, " + + "`elevation` DOUBLE, " + + "`velocity` DOUBLE, " + + "`heading` DOUBLE, " + + "`grade` DOUBLE, " + + "`fuel_consumption` DOUBLE) " + + "TAGS (" + + "`name` VARCHAR(30), " + + "`fleet` VARCHAR(30), " + + "`driver` VARCHAR(30), " + + "`model` VARCHAR(30), " + + "`device_version` VARCHAR(30), " + + "`load_capacity` DOUBLE, " + + "`fuel_capacity` DOUBLE, " + + "`nominal_fuel_consumption` DOUBLE)" + +const DiagnosticsSql = "CREATE STABLE `diagnostics` (" + + "`ts` TIMESTAMP, " + + "`fuel_state` DOUBLE, " + + "`current_load` DOUBLE, " + + "`status` BIGINT) " + + "TAGS (" + + "`name` VARCHAR(30), " + + "`fleet` VARCHAR(30), " + + "`driver` VARCHAR(30), " + + "`model` VARCHAR(30), " + + "`device_version` VARCHAR(30), " + + "`load_capacity` DOUBLE, " + + "`fuel_capacity` DOUBLE, " + + "`nominal_fuel_consumption` DOUBLE)" diff --git a/pkg/targets/timescaledb/creator.go b/pkg/targets/timescaledb/creator.go index ca36181af..195a39be5 100644 --- a/pkg/targets/timescaledb/creator.go +++ b/pkg/targets/timescaledb/creator.go @@ -116,6 +116,7 @@ func (d *dbCreator) PostCreateDB(dbName string) error { } r = MustQuery(dbBench, checkTableQuery) } + return nil } } return nil diff --git a/scripts/full_cycle_minitest/full_cycle_minitest_tdengine.sh b/scripts/full_cycle_minitest/full_cycle_minitest_tdengine.sh new file mode 100755 index 000000000..64de2865a --- /dev/null +++ b/scripts/full_cycle_minitest/full_cycle_minitest_tdengine.sh @@ -0,0 +1,25 @@ +#!/bin/bash +# showcases the ftsb 3 phases for TDengine +# - 1) data and query generation +# - 2) data loading/insertion +# - 3) query execution + +MAX_QUERIES=${MAX_QUERIES:-"1000"} + +mkdir -p /tmp/bulk_data + +# generate data +$GOPATH/bin/tsbs_generate_data --format TDengine --use-case cpu-only --scale 10 --seed 123 --file /tmp/bulk_data/tdengine_data + +# generate queries +$GOPATH/bin/tsbs_generate_queries --queries=${MAX_QUERIES} --format TDengine --use-case cpu-only --scale 10 --seed 123 --query-type lastpoint --file /tmp/bulk_data/tdengine_query_lastpoint +$GOPATH/bin/tsbs_generate_queries --queries=${MAX_QUERIES} --format TDengine --use-case cpu-only --scale 10 --seed 123 --query-type cpu-max-all-1 --file /tmp/bulk_data/tdengine_query_cpu-max-all-1 +$GOPATH/bin/tsbs_generate_queries --queries=${MAX_QUERIES} --format TDengine --use-case cpu-only --scale 10 --seed 123 --query-type high-cpu-1 --file /tmp/bulk_data/tdengine_query_high-cpu-1 + +# insert benchmark +$GOPATH/bin/tsbs_load_tdengine --db-name=benchmark --workers=1 --file=/tmp/bulk_data/tdengine_data --results-file="tdengine_load_results.json" + +# queries benchmark +$GOPATH/bin/tsbs_run_queries_tdengine --db-name=benchmark --workers=1 --max-queries=${MAX_QUERIES} --file=/tmp/bulk_data/tdengine_query_lastpoint --results-file="tdengine_query_lastpoint_results.json" +$GOPATH/bin/tsbs_run_queries_tdengine --db-name=benchmark --workers=1 --max-queries=${MAX_QUERIES} --file=/tmp/bulk_data/tdengine_query_cpu-max-all-1 --results-file="tdengine_query_cpu-max-all-1_results.json" +$GOPATH/bin/tsbs_run_queries_tdengine --db-name=benchmark --workers=1 --max-queries=${MAX_QUERIES} --file=/tmp/bulk_data/tdengine_query_high-cpu-1 --results-file="tdengine_query_high-cpu-1_results.json" \ No newline at end of file diff --git a/scripts/load/load_tdengine.sh b/scripts/load/load_tdengine.sh new file mode 100755 index 000000000..5c39deb2e --- /dev/null +++ b/scripts/load/load_tdengine.sh @@ -0,0 +1,42 @@ +#!/bin/bash + +# Ensure loader is available +EXE_FILE_NAME_LOAD_DATA=${EXE_FILE_NAME_LOAD_DATA:-$(which tsbs_load_tdengine)} +if [[ -z "$EXE_FILE_NAME_LOAD_DATA" ]]; then + echo "tsbs_load_tdengine is not available. It is not specified explicitly and not found in \$PATH" + exit 1 +fi + +# Load parameters - common +DATA_FILE_NAME=${DATA_FILE_NAME:-TDengine-data.gz} +DATABASE_NAME=${DATABASE_NAME:-benchmark} +DATABASE_HOST=${DATABASE_HOST:-localhost} +DATABASE_TAOS_PORT=${DATABASE_TAOS_PORT:-6030} +DATABASE_TAOS_PWD=${DATABASE_TAOS_PWD:-taosdata} + +# Load parameters - personal +VGROUPS=${VGROUPS:-"2"} +BUFFER=${BUFFER:-"256"} +PAGES=${PAGES:-"256"} +TRIGGER=${TRIGGER:-"1"} +WALFSYNCPERIOD=${WALFSYNCPERIOD:-"3000"} +WAL_LEVEL=${WAL_LEVEL:-"1"} +HASH_WORKERS=${HASH_WORKERS:-false} + +EXE_DIR=${EXE_DIR:-$(dirname $0)} +source ${EXE_DIR}/load_common.sh + +cat ${DATA_FILE} | gunzip | $EXE_FILE_NAME_LOAD_DATA \ + --db-name=${DATABASE_NAME} \ + --host=${DATABASE_HOST} \ + --port=${DATABASE_TAOS_PORT} \ + --pass=${DATABASE_TAOS_PWD} \ + --workers=${NUM_WORKERS} \ + --batch-size=${BATCH_SIZE} \ + --vgroups=${VGROUPS} \ + --buffer=${BUFFER} \ + --pages=${PAGES} \ + --hash-workers=${HASH_WORKERS} \ + --stt_trigger=${TRIGGER} \ + --wal_level=${WAL_LEVEL} \ + --wal_fsync_period=${WALFSYNCPERIOD} diff --git a/scripts/load/load_tdenginestmt2.sh b/scripts/load/load_tdenginestmt2.sh new file mode 100755 index 000000000..531c98ae2 --- /dev/null +++ b/scripts/load/load_tdenginestmt2.sh @@ -0,0 +1,42 @@ +#!/bin/bash + +# Ensure loader is available +EXE_FILE_NAME_LOAD_DATA=${EXE_FILE_NAME_LOAD_DATA:-$(which tsbs_load_tdenginestmt2)} +if [[ -z "$EXE_FILE_NAME_LOAD_DATA" ]]; then + echo "tsbs_load_tdenginestmt2 is not available. It is not specified explicitly and not found in \$PATH" + exit 1 +fi + +# Load parameters - common +DATA_FILE_NAME=${DATA_FILE_NAME:-TDengineStmt2-data.gz} +DATABASE_NAME=${DATABASE_NAME:-benchmark} +DATABASE_HOST=${DATABASE_HOST:-localhost} +DATABASE_TAOS_PORT=${DATABASE_TAOS_PORT:-6030} +DATABASE_TAOS_PWD=${DATABASE_TAOS_PWD:-taosdata} + +# Load parameters - personal +VGROUPS=${VGROUPS:-"2"} +BUFFER=${BUFFER:-"256"} +PAGES=${PAGES:-"256"} +TRIGGER=${TRIGGER:-"1"} +WALFSYNCPERIOD=${WALFSYNCPERIOD:-"3000"} +WAL_LEVEL=${WAL_LEVEL:-"1"} +HASH_WORKERS=${HASH_WORKERS:-false} + +EXE_DIR=${EXE_DIR:-$(dirname $0)} +source ${EXE_DIR}/load_common.sh + +cat ${DATA_FILE} | gunzip | $EXE_FILE_NAME_LOAD_DATA \ + --db-name=${DATABASE_NAME} \ + --host=${DATABASE_HOST} \ + --port=${DATABASE_TAOS_PORT} \ + --pass=${DATABASE_TAOS_PWD} \ + --workers=${NUM_WORKERS} \ + --batch-size=${BATCH_SIZE} \ + --vgroups=${VGROUPS} \ + --buffer=${BUFFER} \ + --pages=${PAGES} \ + --hash-workers=${HASH_WORKERS} \ + --stt_trigger=${TRIGGER} \ + --wal_level=${WAL_LEVEL} \ + --wal_fsync_period=${WALFSYNCPERIOD} diff --git a/scripts/run_queries/run_queries_tdengine.sh b/scripts/run_queries/run_queries_tdengine.sh new file mode 100755 index 000000000..d0bc0ba00 --- /dev/null +++ b/scripts/run_queries/run_queries_tdengine.sh @@ -0,0 +1,62 @@ +#!/bin/bash + +# Ensure runner is available +EXE_FILE_NAME=${EXE_FILE_NAME:-$(which tsbs_run_queries_tdengine)} +if [[ -z "$EXE_FILE_NAME" ]]; then + echo "tsbs_run_queries_tdengine not available. It is not specified explicitly and not found in \$PATH" + exit 1 +fi + +# Default queries folder +BULK_DATA_DIR=${BULK_DATA_DIR:-"/tmp/bulk_queries"} +MAX_QUERIES=${MAX_QUERIES:-"0"} +# How many concurrent worker would run queries - match num of cores, or default to 4 +NUM_WORKERS=${NUM_WORKERS:-$(grep -c ^processor /proc/cpuinfo 2> /dev/null || echo 4)} + +# +# Run test for one file +# +function run_file() +{ + # $FULL_DATA_FILE_NAME: /full/path/to/file_with.ext + # $DATA_FILE_NAME: file_with.ext + # $DIR: /full/path/to + # $EXTENSION: ext + # NO_EXT_DATA_FILE_NAME: file_with + FULL_DATA_FILE_NAME=$1 + DATA_FILE_NAME=$(basename -- "${FULL_DATA_FILE_NAME}") + DIR=$(dirname "${FULL_DATA_FILE_NAME}") + EXTENSION="${DATA_FILE_NAME##*.}" + NO_EXT_DATA_FILE_NAME="${DATA_FILE_NAME%.*}" + + # Several options on how to name results file + #OUT_FULL_FILE_NAME="${DIR}/result_${DATA_FILE_NAME}" + OUT_FULL_FILE_NAME="${DIR}/result_${NO_EXT_DATA_FILE_NAME}.out" + #OUT_FULL_FILE_NAME="${DIR}/${NO_EXT_DATA_FILE_NAME}.out" + + if [ "${EXTENSION}" == "gz" ]; then + GUNZIP="gunzip" + else + GUNZIP="cat" + fi + + echo "Running ${DATA_FILE_NAME}" + cat $FULL_DATA_FILE_NAME \ + | $GUNZIP \ + | $EXE_FILE_NAME \ + --max-queries $MAX_QUERIES \ + --workers $NUM_WORKERS \ + | tee $OUT_FULL_FILE_NAME +} + +if [ "$#" -gt 0 ]; then + echo "Have $# files specified as params" + for FULL_DATA_FILE_NAME in "$@"; do + run_file $FULL_DATA_FILE_NAME + done +else + echo "Do not have any files specified - run from default queries folder as ${BULK_DATA_DIR}/queries_TDengine*" + for FULL_DATA_FILE_NAME in "${BULK_DATA_DIR}/queries_TDengine"*; do + run_file $FULL_DATA_FILE_NAME + done +fi From 9c8ebebb8cfa13169b16d591472b3e54008eadc4 Mon Sep 17 00:00:00 2001 From: minhuinie Date: Tue, 27 May 2025 11:21:28 +0800 Subject: [PATCH 2/2] use go version 1.14 --- go.mod | 47 ++++---------- go.sum | 195 +++++++++++++++++---------------------------------------- 2 files changed, 71 insertions(+), 171 deletions(-) diff --git a/go.mod b/go.mod index e47a8008c..8d4f0a0cf 100644 --- a/go.mod +++ b/go.mod @@ -5,56 +5,35 @@ go 1.14 require ( github.com/HdrHistogram/hdrhistogram-go v1.0.0 github.com/SiriDB/go-siridb-connector v0.0.0-20190110105621-86b34c44c921 - github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46 // indirect - github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15 // indirect + github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d // indirect github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883 - github.com/aws/aws-sdk-go v1.38.69 - github.com/bitly/go-hostpool v0.1.0 // indirect + github.com/aws/aws-sdk-go v1.35.13 github.com/blagojts/viper v1.6.3-0.20200313094124-068f44cf5e69 - github.com/frankban/quicktest v1.11.3 // indirect - github.com/fsnotify/fsnotify v1.5.1 // indirect github.com/globalsign/mgo v0.0.0-20181015135952-eeefdecb41b8 github.com/go-ole/go-ole v1.2.4 // indirect - github.com/go-sql-driver/mysql v1.6.0 // indirect github.com/gocql/gocql v0.0.0-20190810123941-df4b9cc33030 - github.com/gofrs/uuid v3.3.0+incompatible // indirect - github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang/protobuf v1.5.2 - github.com/golang/snappy v0.0.3 - github.com/google/flatbuffers v2.0.0+incompatible - github.com/google/go-cmp v0.5.6 + github.com/golang/protobuf v1.4.2 + github.com/golang/snappy v0.0.1 + github.com/google/flatbuffers v1.11.0 + github.com/google/go-cmp v0.5.2 github.com/jackc/pgx/v4 v4.8.0 github.com/jmoiron/sqlx v1.2.1-0.20190826204134-d7d95172beb5 github.com/kshvakov/clickhouse v1.3.11 github.com/lib/pq v1.3.0 - github.com/magiconair/properties v1.8.5 // indirect - github.com/mattn/go-sqlite3 v1.14.6 // indirect - github.com/mitchellh/mapstructure v1.4.2 // indirect - github.com/pelletier/go-toml v1.9.4 // indirect - github.com/pierrec/lz4 v2.5.2+incompatible // indirect github.com/pkg/errors v0.9.1 - github.com/prometheus/common v0.26.0 - github.com/shirou/gopsutil v3.21.8+incompatible - github.com/shopspring/decimal v1.2.0 // indirect + github.com/prometheus/common v0.13.0 + github.com/shirou/gopsutil v3.21.3+incompatible github.com/silenceper/pool v1.0.0 - github.com/sirupsen/logrus v1.8.1 // indirect github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 - github.com/spf13/afero v1.6.0 // indirect - github.com/spf13/cast v1.4.1 // indirect github.com/spf13/cobra v1.0.0 - github.com/spf13/jwalterweatherman v1.1.0 // indirect github.com/spf13/pflag v1.0.5 github.com/taosdata/driver-go/v3 v3.6.0 github.com/timescale/promscale v0.0.0-20201006153045-6a66a36f5c84 github.com/tklauser/go-sysconf v0.3.5 // indirect github.com/transceptor-technology/go-qpack v0.0.0-20190116123619-49a14b216a45 - github.com/valyala/fasthttp v1.34.0 - go.uber.org/atomic v1.7.0 - golang.org/x/net v0.7.0 - golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac - google.golang.org/genproto v0.0.0-20210921142501-181ce0d877f6 // indirect - gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect - gopkg.in/ini.v1 v1.63.2 // indirect - gopkg.in/yaml.v2 v2.4.0 - gopkg.in/yaml.v3 v3.0.0 // indirect + github.com/valyala/fasthttp v1.15.1 + go.uber.org/atomic v1.6.0 + golang.org/x/net v0.0.0-20200904194848-62affa334b73 + golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e + gopkg.in/yaml.v2 v2.3.0 ) diff --git a/go.sum b/go.sum index fe4275d6c..c7d7ca04b 100644 --- a/go.sum +++ b/go.sum @@ -46,6 +46,7 @@ github.com/Azure/go-autorest/logger v0.1.0/go.mod h1:oExouG+K6PryycPJfVSxi/koC6L github.com/Azure/go-autorest/logger v0.2.0/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZmbF5NWuPV8+WeEW8= github.com/Azure/go-autorest/tracing v0.5.0/go.mod h1:r/s2XiOKccPW3HrqB+W0TQzfbtp2fGCgRFtBroKn4Dk= github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= +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/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= @@ -73,8 +74,8 @@ github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMx github.com/SiriDB/go-siridb-connector v0.0.0-20190110105621-86b34c44c921 h1:GIWNb0z3t/YKr7xcGNhFgxasaTpnsX91Z0Zt4CeLk+c= github.com/SiriDB/go-siridb-connector v0.0.0-20190110105621-86b34c44c921/go.mod h1:s0x47OhsrJKfg9Iq5orGCVJQjwKklC3jZMFlgLe6Zew= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= -github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46 h1:5sXbqlSomvdjlRbWyNqkPsJ3Fg+tQZCbgeX1VGljbQY= -github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= +github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= 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/agnivade/levenshtein v1.0.1/go.mod h1:CURSv5d9Uaml+FovSIICkLbAUZ9S4RqaHDIsdSBg7lM= @@ -84,14 +85,12 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafo 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 h1:UQZhZ2O0vMHr2cI+DC1Mbh0TJxzA3RcLoMsFw+aXw7E= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= -github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15 h1:AUNCr9CiJuwrRYS3XieqF+Z9B9gNxo/eANAJCF2eiN4= -github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883 h1:bvNMNQO63//z+xNgfBlViaCIJKLlCJ6/fmUseuG0wVQ= github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883/go.mod h1:rCTlJbsFo29Kk6CurOXKm700vrz8f0KW0JNfpkRJY/8= +github.com/andybalholm/brotli v1.0.0 h1:7UCwP93aiSfvWpapti8g88vVVGp2qqtGyePsSuDafo4= github.com/andybalholm/brotli v1.0.0/go.mod h1:loMXtMfwqflxFJPmdbJO0a3KNoPuLBgiu3qAvBg8x/Y= -github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY= -github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db/go.mod h1:VTxUBvSJ3s3eHAg65PNgrsn5BtqCRPdmyXh6rAfdxN0= github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= @@ -109,16 +108,15 @@ github.com/asaskevich/govalidator v0.0.0-20200108200545-475eaeb16496/go.mod h1:o 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 v1.34.9/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= -github.com/aws/aws-sdk-go v1.38.69 h1:V489lmrdkIQSfF6OAGZZ1Cavcm7eczCm2JcGvX+yHRg= -github.com/aws/aws-sdk-go v1.38.69/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= +github.com/aws/aws-sdk-go v1.35.13 h1:Y49GifH2czbooBMkVpoXwokur1JRBFKVLVCQzO0YsW8= +github.com/aws/aws-sdk-go v1.35.13/go.mod h1:tlPOdRjfxPBpNIwqDj61rmsnA85v9jc0Ps9+muhnW+k= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= 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/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932 h1:mXoPYz/Ul5HYEDvkta6I8/rnYM5gSdSV2tJ6XbZuEtY= github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932/go.mod h1:NOuUCSz6Q9T7+igc/hlvDOUdtWKryOrtFyIVABv/p7k= -github.com/bitly/go-hostpool v0.1.0 h1:XKmsF6k5el6xHG3WPJ8U0Ku/ye7njX7W81Ng7O2ioR0= -github.com/bitly/go-hostpool v0.1.0/go.mod h1:4gOCgp6+NZnVqlKyZ/iBZFTAJKembaVENUpMkpg42fw= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= @@ -147,8 +145,6 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 h1:F1EaeKL/ta07PY/k9Os/UFtwERei2/XzGemhpGnBKNg= github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58/go.mod h1:EOBUe0h4xcZ5GoxqC5SDxFQ8gwyZPKQoEzownBlhI80= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/apd v1.1.0 h1:3LFP3629v+1aKXU5Q37mxmRxX/pIu1nijXydLShEq5I= github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= @@ -201,8 +197,6 @@ github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4s github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/evanphx/json-patch v0.0.0-20200808040245-162e5629780b/go.mod h1:NAJj0yf/KaRKURN6nyi7A9IZydMivZEm9oQLWNjfKDc= github.com/evanphx/json-patch v4.2.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= @@ -211,12 +205,9 @@ github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= 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/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= -github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= -github.com/fsnotify/fsnotify v1.5.1 h1:mZcQUHVQUQWoPXXtuf9yuEXKudkV2sx1E06UadKWpgI= -github.com/fsnotify/fsnotify v1.5.1/go.mod h1:T3375wBYaZdLLcVNkcVbzGHY7f1l/uK5T5Ai1i3InKU= 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/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= @@ -233,7 +224,6 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2 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-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= 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= @@ -306,9 +296,8 @@ github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GO github.com/go-redis/redis v6.15.7+incompatible/go.mod h1:NAIEuMOZ/fxfXJIrKDQDz8wamY7mA7PouImQ2Jvg6kA= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= +github.com/go-sql-driver/mysql v1.5.0 h1:ozyZYNQW3x3HtqT1jira07DN2PArx2v7/mN66gGcHOs= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= -github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-toolsmith/astcast v1.0.0/go.mod h1:mt2OdQTeAQcY4DQgPSArJjHCcOwlX+Wl/kwN+LbLGQ4= github.com/go-toolsmith/astcopy v1.0.0/go.mod h1:vrgyG+5Bxrnz4MZWPF+pI4R8h3qKRjjyvV/DSez4WVQ= @@ -349,16 +338,14 @@ github.com/gobwas/glob v0.2.3/go.mod h1:d3Ez4x06l9bZtSvzIay5+Yzi0fmZzPgnTbPcKjJA github.com/gocql/gocql v0.0.0-20190810123941-df4b9cc33030 h1:mqUk3AueyxYmzrE0nu29YlwjmuaWt2sUwk+CrTEGbmY= github.com/gocql/gocql v0.0.0-20190810123941-df4b9cc33030/go.mod h1:Q7Sru5153KG8D9zwueuQJB3ccJf9/bIwF/x8b3oKgT8= github.com/gofrs/flock v0.8.0/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= +github.com/gofrs/uuid v3.2.0+incompatible h1:y12jRkkFxsd7GpqdSZ+/KCs/fJbqpEXSGd4+jfEaewE= github.com/gofrs/uuid v3.2.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= -github.com/gofrs/uuid v3.3.0+incompatible h1:8K4tyRfvU1CYPgJsveYFQMhpFd/wXNM7iK6rR7UHz84= -github.com/gofrs/uuid v3.3.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= github.com/gogo/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.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= -github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -384,16 +371,12 @@ github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrU 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.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2 h1:+Z5KGCizgyZCbGh1KZqA0fcLLkwbsjIzS4aV2v7wJX0= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.0-20170215233205-553a64147049/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA= -github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2/go.mod h1:k9Qvh+8juN+UKMCS/3jFtGICgW8O96FVaZsaxdzDkR4= github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a/go.mod h1:ryS0uhF+x9jgbj/N71xsEqODy9BN81/GonCZiOzirOk= github.com/golangci/errcheck v0.0.0-20181223084120-ef45e06d44b6/go.mod h1:DbHgvLiFKX1Sh2T1w8Q/h4NAI8MHIpzCdnBUDTXU3I0= @@ -411,19 +394,15 @@ github.com/golangci/revgrep v0.0.0-20180526074752-d9c87f5ffaf0/go.mod h1:qOQCunE github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4/go.mod h1:Izgrg8RkN3rCIMLGE9CyYmU9pY2Jer6DgANEnZ/L/cQ= 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/flatbuffers v1.11.0 h1:O7CEyB8Cb3/DmtxODGtLHcEvpr81Jm5qLg/hsHnxA2A= github.com/google/flatbuffers v1.11.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= -github.com/google/flatbuffers v2.0.0+incompatible h1:dicJ2oXwypfwUGnB2/TYWYEKiuk9eYQlQO/AnOHl5mI= -github.com/google/flatbuffers v2.0.0+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= 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/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.6 h1:BKbKCqvP6I+rmFHt06ZmyQtvB8xAkWdhFyr0ZUNZcxQ= -github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.1.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= @@ -437,7 +416,6 @@ github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hf 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/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= @@ -448,6 +426,7 @@ github.com/googleapis/gnostic v0.4.0/go.mod h1:on+2t9HRStVgn95RSsFWFz+6Q0Snyqv1a github.com/gookit/color v1.2.5/go.mod h1:AhIE+pS6D4Ql0SQWbBeXPHw7gY0/sjHoA4s/n1KB7xg= github.com/gophercloud/gophercloud v0.1.0/go.mod h1:vxM41WHh5uqHVBMZHzuwNOHh8XEoIEcSTewFxm1c5g8= github.com/gophercloud/gophercloud v0.12.0/go.mod h1:gmC5oQqMDOMO1t1gq5DquX/yAU808e/4mzjjDA76+Ss= +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= @@ -466,9 +445,8 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgf 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/grpc-ecosystem/grpc-gateway v1.14.6/go.mod h1:zdiPV4Yse/1gnckTHtghG4GkDEdKCRJduHpTxT3/jcw= +github.com/grpc-ecosystem/grpc-gateway v1.14.8 h1:hXClj+iFpmLM8i3lkO6i4Psli4P2qObQuQReiII26U8= github.com/grpc-ecosystem/grpc-gateway v1.14.8/go.mod h1:NZE8t6vs6TnwLL/ITkaK8W3ecMLGAbh2jXTclvpiwYo= -github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= -github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed h1:5upAirOpQc1Q53c0bnx2ufif5kANL7bfZWcc6VJWJd8= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed/go.mod h1:tMWxXQ9wFIaZeTI9F+hmhFiGpFmhOHzyShyFUhRm0H4= github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= @@ -603,6 +581,7 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jsternberg/zap-logfmt v1.0.0/go.mod h1:uvPs/4X51zdkcm5jXl5SYoN+4RK21K8mysFmDaM/h+o= +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= @@ -612,26 +591,23 @@ github.com/karrick/godirwalk v1.8.0/go.mod h1:H5KPZjojv4lE+QYImBI8xVtrBRgYrIVsaR github.com/karrick/godirwalk v1.10.3/go.mod h1:RoGL9dQei4vP9ilrpETWE8CLOZ1kiN0LhBygSwrAsHA= 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/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.10.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/compress v1.10.10 h1:a/y8CglcM7gLGYmlbP/stPE5sR3hbhFRUjCBfd/0B3I= github.com/klauspost/compress v1.10.10/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/compress v1.15.0 h1:xqfchp4whNFxn5A4XFyyYtitiWI8Hy5EW59jEwcyL6U= -github.com/klauspost/compress v1.15.0/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPRLkC4NPOvfYeR5KNOrY6TD+/sAC3HXPZgDYg= github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/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.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/fs v0.1.0/go.mod h1:FFnZGqtBN9Gxj7eW1uZ42v5BccTP0vu6NEaFoC2HwRg= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= -github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA= github.com/kr/pty v1.1.8/go.mod h1:O1sed60cT9XZ5uDucP5qwvh+TE3NnUj51EiZO/lmSfw= @@ -653,9 +629,8 @@ github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0U github.com/logrusorgru/aurora v0.0.0-20181002194514-a7b3b318ed4e/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4= 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/magiconair/properties v1.8.5 h1:b6kJs+EmPFMYGkow9GiUyCyOvIwYetYJ3fSaWak/Gls= -github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= github.com/mailru/easyjson v0.0.0-20160728113105-d5b7844b561a/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -685,9 +660,8 @@ github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Ky github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/mattn/go-sqlite3 v1.11.0 h1:LDdKkqtYlom37fkvqs8rMPFKAMe8+SgjbwZ6ex1/A/Q= github.com/mattn/go-sqlite3 v1.11.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= -github.com/mattn/go-sqlite3 v1.14.6 h1:dNPt6NO46WmLVt2DLNpwczCmdV5boIZ6g/tlDrlRUbg= -github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE= github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -705,9 +679,8 @@ github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS4 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/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mitchellh/mapstructure v1.2.2 h1:dxe5oCinTXiTIcfgmZecdCzPmAJKd46KsCWc35r0TV4= github.com/mitchellh/mapstructure v1.2.2/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= -github.com/mitchellh/mapstructure v1.4.2 h1:6h7AQ0yhTcIsmFmnAwQls75jp2Gzs4iB8W7pjMO+rqo= -github.com/mitchellh/mapstructure v1.4.2/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= 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= @@ -733,6 +706,7 @@ github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxzi 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/nbutton23/zxcvbn-go v0.0.0-20180912185939-ae427f1e4c1d/go.mod h1:o96djdrsSGy3AWPyBgZMAGfxZNfgntdJG+11KU4QvbU= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nishanths/exhaustive v0.0.0-20200811152831-6cf413ae40e0/go.mod h1:wBEpHwM2OdmeNpdCvRPUlkEbBuaFmcK4Wv8Q7FuGW3c= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= @@ -776,24 +750,21 @@ github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144T github.com/paulbellamy/ratecounter v0.2.0/go.mod h1:Hfx1hDpSGoqxkVVpBi/IlYD7kChlfo5C6hzIHwPqfFE= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= +github.com/pelletier/go-toml v1.4.0 h1:u3Z1r+oOXJIkxqw34zVhyPgjBsm6X2wn21NWs/HfSeg= github.com/pelletier/go-toml v1.4.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= -github.com/pelletier/go-toml v1.9.4 h1:tjENF6MfZAg8e4ZmZTeWaWiT2vXtsoO6+iuOjFhECwM= -github.com/pelletier/go-toml v1.9.4/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= github.com/peterbourgon/diskv v2.0.1+incompatible/go.mod h1:uqqh8zWWbv1HBMNONnaR/tNboyR3/BZd58JJSHlUSCU= github.com/peterh/liner v1.0.1-0.20180619022028-8c1271fcf47f/go.mod h1:xIteQHvHuaLYG9IFj6mSxM0fCKrs34IrEQUhOYuGPHc= github.com/phayes/checkstyle v0.0.0-20170904204023-bfd46e6a821d/go.mod h1:3OzsM7FXDQlpCiw2j81fOmAwQLnZnLGXVKUzeKQXIAw= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc= +github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4 v2.5.2+incompatible h1:WCjObylUIOlKy/+7Abdn34TLIkXiA4UWUMhxq9m9ZXI= -github.com/pierrec/lz4 v2.5.2+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 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= 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/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZI= github.com/pkg/term v0.0.0-20180730021639-bffc007b7fd5/go.mod h1:eCbImbZ95eXtAUIbLAuAVnBnwf83mjf6QIVH8SHYwqQ= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= @@ -822,9 +793,8 @@ github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+ github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.13.0 h1:vJlpe9wPgDRM1Z+7Wj3zUUjY1nr6/1jNKyl7llliccg= github.com/prometheus/common v0.13.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= -github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= -github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= 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= @@ -867,13 +837,13 @@ github.com/sergi/go-diff v1.0.0 h1:Kpca3qRNrduNnOQeazBd0ysaKrUJiIuISHxogkT9RPQ= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shazow/go-diff v0.0.0-20160112020656-b6b7b6733b8c/go.mod h1:/PevMnwAxekIXwN8qQyfc5gl2NlkB3CQlkizAbOkeBs= github.com/shirou/gopsutil v0.0.0-20190901111213-e4ec7b275ada/go.mod h1:WWnYX4lzhCH5h/3YBfyVA3VbLYjlMZZAQcW9ojMexNc= -github.com/shirou/gopsutil v3.21.8+incompatible h1:sh0foI8tMRlCidUJR+KzqWYWxrkuuPIGiO6Vp+KXdCU= -github.com/shirou/gopsutil v3.21.8+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/gopsutil v3.21.3+incompatible h1:uenXGGa8ESCQq+dbgtl916dmg6PSAz2cXov0uORQ9v8= +github.com/shirou/gopsutil v3.21.3+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4 h1:udFKJ0aHUL60LboW/A+DfgoHVedieIzIXE8uylPue0U= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= +github.com/shopspring/decimal v0.0.0-20200227202807-02e2044944cc h1:jUIKcSPO9MoMJBbEoyE/RJoE8vz7Mb8AjvifMMwSyvY= github.com/shopspring/decimal v0.0.0-20200227202807-02e2044944cc/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= -github.com/shopspring/decimal v1.2.0 h1:abSATXmQEYyShuxI4/vyW3tV1MrKAJzCZ/0zLUXYbsQ= -github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= github.com/shurcooL/go v0.0.0-20180423040247-9e1955d9fb6e/go.mod h1:TDJrrUr11Vxrven61rcy3hJMUqaf/CLWYhHNPmT14Lk= github.com/shurcooL/go-goon v0.0.0-20170922171312-37c2f522c041/go.mod h1:N5mDOmsrJOB+vfqUK+7DmDyjhSLIIBnXo9lvZJj3MWQ= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= @@ -886,10 +856,11 @@ github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPx github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= -github.com/sirupsen/logrus v1.8.1/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/sonatard/noctx v0.0.1/go.mod h1:9D2D/EoULe8Yy2joDHJj7bv3sZoq9AaSb8B4lqBjiZI= @@ -898,19 +869,16 @@ github.com/sourcegraph/go-diff v0.6.0/go.mod h1:iBszgVvyxdc8SFZ7gm69go2KDdt3ag07 github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= +github.com/spf13/afero v1.2.2 h1:5jhuqJyZCZf2JRofRvN/nIFgIWNzPa3/Vz8mYylgbWc= github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= -github.com/spf13/afero v1.6.0 h1:xoax2sJ2DT8S8xA2paPFjDCScCNeWsg75VG0DLRreiY= -github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= +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/cast v1.4.1 h1:s0hze+J0196ZfEMTs80N7UlFt0BDuQ7Q+JDnHiMWKdA= -github.com/spf13/cast v1.4.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.7/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= +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/jwalterweatherman v1.1.0 h1:ue6voC5bR5F8YxI5S67j9i582FU4Qvo2bmqnqMYADFk= -github.com/spf13/jwalterweatherman v1.1.0/go.mod h1:aNWZUN0dPAAO/Ljvb5BEdw96iTZ0EXowPYD95IqWIGo= 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= @@ -930,6 +898,7 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf 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.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +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/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= @@ -967,12 +936,10 @@ github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtX github.com/uudashr/gocognit v1.0.1/go.mod h1:j44Ayx2KW4+oB6SWMv8KsmHzZrOInQav7D3cQMJ5JUM= 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.15.1 h1:eRb5jzWhbCn/cGu3gNJMcOfPUfXgXCcQIOHjh9ajAS8= github.com/valyala/fasthttp v1.15.1/go.mod h1:YOKImeEosDdBPnxc0gy7INqi3m1zK6A+xl6TwOBhHCA= -github.com/valyala/fasthttp v1.34.0 h1:d3AAQJ2DRcxJYHm7OXNXtXt2as1vMDfxeIcFvhmGGm4= -github.com/valyala/fasthttp v1.34.0/go.mod h1:epZA5N+7pY6ZaEKRmstzOuYJx9HI8DI1oaCGZpdH4h0= github.com/valyala/quicktemplate v1.6.2/go.mod h1:mtEJpQtUiBV0SHhMX6RtiJtqxncgrfmjcUy5T68X8TM= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= -github.com/valyala/tcplisten v1.0.0/go.mod h1:T0xQ8SeCZGxckz9qRXTfG43PvQ/mcWh7FwZEA7Ioqkc= github.com/vektah/gqlparser v1.1.2/go.mod h1:1ycwN7Ij5njmMkPPAOaRFY4rET2Enx7IkVv3vaXspKw= github.com/willf/bitset v1.1.3/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= @@ -985,7 +952,6 @@ github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= @@ -1001,13 +967,11 @@ 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.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= 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/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= @@ -1037,10 +1001,8 @@ golang.org/x/crypto v0.0.0-20191202143827-86a70503ff7e/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200220183623-bac4c82f6975/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200323165209-0ec3e9974c59/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rBCcS0QyQY66Mpf/7BZbInM= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220214200702-86341886e292 h1:f+lwQ+GtmgoY+A2YaQxlSOnDjXcQ7ZRLWOHbC6HtRqE= -golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1066,6 +1028,7 @@ golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= 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= @@ -1074,8 +1037,8 @@ 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/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= 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= @@ -1115,16 +1078,8 @@ golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/ golang.org/x/net v0.0.0-20200602114024-627f9648deb9/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200904194848-62affa334b73 h1:MXfv8rhZWmFeqX3GNZRsd6vOLoaCHjYEX3qkRo3YBUA= golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= -golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= 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= @@ -1139,8 +1094,6 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/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= @@ -1198,42 +1151,23 @@ golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7w 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-20200821140526-fda516888d29/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200908134130-d2e65c121b96 h1:gJciq3lOg0eS9fSZJcoHfv7q1BfC6cJfnmSSKL1yu3Q= golang.org/x/sys v0.0.0-20200908134130-d2e65c121b96/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210316164454-77fc1eacc6aa h1:ZYxPR6aca/uhfRJyaOAtflSHjJYiktO7QnJC5ut7iY4= golang.org/x/sys v0.0.0-20210316164454-77fc1eacc6aa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220227234510-4e6760a101f9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= -golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/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/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= -golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= 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/time v0.0.0-20200630173020-3af7569d3a1e h1:EHBhcS0mlXEAVwNyO2dLfjToGsyY4j24pTs2ScHnX7s= golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac h1:7zkz7BUtwNFFqcowJ+RIgu2MaV/MapERkDIy+mwPyjs= -golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180810170437-e96c4e24768d/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1303,16 +1237,14 @@ golang.org/x/tools v0.0.0-20200414032229-332987a829c3/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200422022333-3d57cf2e726e/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200513201620-d5fe73897c97/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200519015757-0d0afa43d58a/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200625211823-6506e20df31f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200626171337-aa94e735be7f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200701041122-1837592efa10/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200724022722-7017fd6b1305/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200812195022-5ae4c3c160a0/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200822203824-307de81be3f4/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200908211811-12e1bf57a112 h1:DmrRJy1qn9VDMf4+GSpRlwfZ51muIF7r96MFBFP4bPM= golang.org/x/tools v0.0.0-20200908211811-12e1bf57a112/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190513163551-3ee3066db522/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1369,9 +1301,8 @@ google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfG google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20200815001618-f69a88009b70/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d h1:92D1fum1bJLKSdr11OJ+54YeCMCGYIygTA7R/YZxH5M= google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210921142501-181ce0d877f6 h1:2ncG/LajxmrclaZH+ppVi02rQxz4eXYJzGHdFN4Y9UA= -google.golang.org/genproto v0.0.0-20210921142501-181ce0d877f6/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= 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= @@ -1388,11 +1319,8 @@ google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.32.0 h1:zWTV+LMdc3kaiJMSTOFz2UgSBgx8RNQoTGiZu3fR9S0= google.golang.org/grpc v1.32.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= -google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.40.0 h1:AGJ0Ih4mHjSeibYkFGh1dD9KJ/eOtZ93I6hoHhukQ5Q= -google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= 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= @@ -1402,19 +1330,15 @@ google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2 google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= +google.golang.org/protobuf v1.25.0 h1:Ejskq+SyPohKW+1uil0JJMtmHCgJPJ/qWTxr8qp+R4c= google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= -google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc= 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/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f h1:BLraFXnmrev5lT+xlilqcH8XK9/i0At2xKjWk4p6zsU= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= @@ -1423,9 +1347,8 @@ gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= gopkg.in/inconshreveable/log15.v2 v2.0.0-20180818164646-67afb5ed74ec/go.mod h1:aPpfJ7XW+gOuirDoZ8gHhLh3kZ1B08FtV2bbmy7Jv3s= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= +gopkg.in/ini.v1 v1.51.0 h1:AQvPpx3LzTDM0AjnIRlVFwFFGC+npRopjZxLJj6gdno= gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= -gopkg.in/ini.v1 v1.63.2 h1:tGK/CyBg7SMzb60vP1M03vNZ3VDu3wGQJwn7Sxi9r3c= -gopkg.in/ini.v1 v1.63.2/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI= @@ -1437,13 +1360,11 @@ 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.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/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.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0 h1:hjy8E9ON/egN1tAYqKb61G10WtihqetD4sz2H+8nIeA= -gopkg.in/yaml.v3 v3.0.0/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools v0.0.0-20181223230014-1083505acf35/go.mod h1:R//lfYlUuTOTfblYI3lGoAAAebUdzjvbmQsuB7Ykd90= gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=