diff --git a/Makefile b/Makefile index bc6c3fbafd056..dced1380341e4 100644 --- a/Makefile +++ b/Makefile @@ -147,10 +147,8 @@ testSuite: @echo "testSuite" ./tools/check/check_testSuite.sh -clean: +clean: failpoint-disable $(GO) clean -i ./... - rm -rf *.out - rm -rf parser # Split tests for CI to run `make test` in parallel. test: test_part_1 test_part_2 diff --git a/cmd/explaintest/r/explain.result b/cmd/explaintest/r/explain.result index 34ba40144e66e..cd87312a0d2dd 100644 --- a/cmd/explaintest/r/explain.result +++ b/cmd/explaintest/r/explain.result @@ -39,3 +39,8 @@ StreamAgg_8 8000.00 root group by:Column#7, funcs:group_concat(Column#5, Column └─TableReader_15 10000.00 root data:TableFullScan_14 └─TableFullScan_14 10000.00 cop[tikv] table:t keep order:true, stats:pseudo drop table t; +drop view if exists v; +create view v as select cast(replace(substring_index(substring_index("",',',1),':',-1),'"','') as CHAR(32)) as event_id; +desc v; +Field Type Null Key Default Extra +event_id varchar(32) YES NULL diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 6f954211f94d0..7b92ecbaeefd8 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -556,28 +556,29 @@ Projection_4 10.00 root plus(1, test.t.nb)->Column#5 explain select * from t ta left outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(ta.nb, 1) or ta.nb is null; id estRows task access object operator info HashJoin_7 8320.83 root left outer join, equal:[eq(test.t.nb, test.t.nb)], left cond:[gt(test.t.a, 1)] -├─TableReader_13(Build) 10000.00 root data:TableFullScan_12 -│ └─TableFullScan_12 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo +├─TableReader_14(Build) 6656.67 root data:Selection_13 +│ └─Selection_13 6656.67 cop[tikv] or(test.t.nb, 0) +│ └─TableFullScan_12 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo └─TableReader_11(Probe) 6656.67 root data:Selection_10 - └─Selection_10 6656.67 cop[tikv] or(test.t.nb, isnull(test.t.nb)) + └─Selection_10 6656.67 cop[tikv] or(test.t.nb, 0) └─TableFullScan_9 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo explain select * from t ta right outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.nb, 1) or tb.nb is null; id estRows task access object operator info HashJoin_7 6656.67 root right outer join, equal:[eq(test.t.nb, test.t.nb)] -├─TableReader_11(Build) 3333.33 root data:Selection_10 -│ └─Selection_10 3333.33 cop[tikv] gt(test.t.a, 1) +├─TableReader_11(Build) 2218.89 root data:Selection_10 +│ └─Selection_10 2218.89 cop[tikv] gt(test.t.a, 1), or(test.t.nb, 0) │ └─TableFullScan_9 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo └─TableReader_14(Probe) 6656.67 root data:Selection_13 - └─Selection_13 6656.67 cop[tikv] or(test.t.nb, isnull(test.t.nb)) + └─Selection_13 6656.67 cop[tikv] or(test.t.nb, 0) └─TableFullScan_12 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo explain select * from t ta inner join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.nb, 1) or tb.nb is null; id estRows task access object operator info -HashJoin_9 4166.67 root inner join, equal:[eq(test.t.nb, test.t.nb)] -├─TableReader_12(Build) 3333.33 root data:Selection_11 -│ └─Selection_11 3333.33 cop[tikv] gt(test.t.a, 1) +HashJoin_9 2773.61 root inner join, equal:[eq(test.t.nb, test.t.nb)] +├─TableReader_12(Build) 2218.89 root data:Selection_11 +│ └─Selection_11 2218.89 cop[tikv] gt(test.t.a, 1), or(test.t.nb, 0) │ └─TableFullScan_10 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo └─TableReader_15(Probe) 6656.67 root data:Selection_14 - └─Selection_14 6656.67 cop[tikv] or(test.t.nb, isnull(test.t.nb)) + └─Selection_14 6656.67 cop[tikv] or(test.t.nb, 0) └─TableFullScan_13 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo explain select ifnull(t.nc, 1) in (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t; id estRows task access object operator info diff --git a/cmd/explaintest/r/partition_pruning.result b/cmd/explaintest/r/partition_pruning.result index cf9b019d374c9..fd5930d18ae1b 100644 --- a/cmd/explaintest/r/partition_pruning.result +++ b/cmd/explaintest/r/partition_pruning.result @@ -2333,8 +2333,8 @@ TableReader_7 250.00 root partition:all data:Selection_6 └─TableFullScan_5 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain select * from t1 where a>=0 and a <= 0xFFFFFFFFFFFFFFFF; id estRows task access object operator info -TableReader_7 250.00 root partition:all data:Selection_6 -└─Selection_6 250.00 cop[tikv] ge(test.t1.a, 0), le(test.t1.a, 18446744073709551615) +TableReader_7 3323.33 root partition:all data:Selection_6 +└─Selection_6 3323.33 cop[tikv] le(test.t1.a, 18446744073709551615) └─TableFullScan_5 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo drop table t1; create table t1 (a bigint) partition by range(a+0) ( diff --git a/cmd/explaintest/t/explain.test b/cmd/explaintest/t/explain.test index 3bde3f5d7d5ca..33fd0e7165d61 100644 --- a/cmd/explaintest/t/explain.test +++ b/cmd/explaintest/t/explain.test @@ -14,3 +14,7 @@ set session tidb_hashagg_final_concurrency = 1; explain select group_concat(a) from t group by id; explain select group_concat(a, b) from t group by id; drop table t; + +drop view if exists v; +create view v as select cast(replace(substring_index(substring_index("",',',1),':',-1),'"','') as CHAR(32)) as event_id; +desc v; diff --git a/config/config.go b/config/config.go index c64576c9b07d5..40049b57d0465 100644 --- a/config/config.go +++ b/config/config.go @@ -38,6 +38,7 @@ import ( tracing "github.com/uber/jaeger-client-go/config" "go.uber.org/zap" + "google.golang.org/grpc/encoding/gzip" ) // Config number limitations @@ -493,6 +494,8 @@ type TiKVClient struct { // After having pinged for keepalive check, the client waits for a duration of Timeout in seconds // and if no activity is seen even after that the connection is closed. GrpcKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"` + // GrpcCompressionType is the compression type for gRPC channel: none or gzip. + GrpcCompressionType string `toml:"grpc-compression-type" json:"grpc-compression-type"` // CommitTimeout is the max time which command 'commit' will wait. CommitTimeout string `toml:"commit-timeout" json:"commit-timeout"` AsyncCommit AsyncCommit `toml:"async-commit" json:"async-commit"` @@ -693,6 +696,7 @@ var defaultConf = Config{ GrpcConnectionCount: 4, GrpcKeepAliveTime: 10, GrpcKeepAliveTimeout: 3, + GrpcCompressionType: "none", CommitTimeout: "41s", AsyncCommit: AsyncCommit{ Enable: false, @@ -882,7 +886,7 @@ func (c *Config) Valid() error { if c.Security.SkipGrantTable && !hasRootPrivilege() { return fmt.Errorf("TiDB run with skip-grant-table need root privilege") } - if _, ok := ValidStorage[c.Store]; !ok { + if !ValidStorage[c.Store] { nameList := make([]string, 0, len(ValidStorage)) for k, v := range ValidStorage { if v { @@ -918,6 +922,9 @@ func (c *Config) Valid() error { if c.TiKVClient.GrpcConnectionCount == 0 { return fmt.Errorf("grpc-connection-count should be greater than 0") } + if c.TiKVClient.GrpcCompressionType != "none" && c.TiKVClient.GrpcCompressionType != gzip.Name { + return fmt.Errorf("grpc-compression-type should be none or %s, but got %s", gzip.Name, c.TiKVClient.GrpcCompressionType) + } if c.Performance.TxnTotalSizeLimit > 10<<30 { return fmt.Errorf("txn-total-size-limit should be less than %d", 10<<30) diff --git a/config/config.toml.example b/config/config.toml.example index 3fdb3863d76bf..fb119a41b75e7 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -353,6 +353,9 @@ grpc-keepalive-time = 10 # and if no activity is seen even after that the connection is closed. grpc-keepalive-timeout = 3 +# The compression type for gRPC channel: none or gzip. +grpc-compression-type = "none" + # Max time for commit command, must be twice bigger than raft election timeout. commit-timeout = "41s" diff --git a/ddl/column.go b/ddl/column.go index 07ece53470b23..cf4d55c20e034 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -516,17 +516,18 @@ func checkDropColumnForStatePublic(tblInfo *model.TableInfo, colInfo *model.Colu // When the dropping column has not-null flag and it hasn't the default value, we can backfill the column value like "add column". // NOTE: If the state of StateWriteOnly can be rollbacked, we'd better reconsider the original default value. // And we need consider the column without not-null flag. - if colInfo.OriginDefaultValue == nil && mysql.HasNotNullFlag(colInfo.Flag) { + if colInfo.GetOriginDefaultValue() == nil && mysql.HasNotNullFlag(colInfo.Flag) { // If the column is timestamp default current_timestamp, and DDL owner is new version TiDB that set column.Version to 1, // then old TiDB update record in the column write only stage will uses the wrong default value of the dropping column. // Because new version of the column default value is UTC time, but old version TiDB will think the default value is the time in system timezone. // But currently will be ok, because we can't cancel the drop column job when the job is running, // so the column will be dropped succeed and client will never see the wrong default value of the dropped column. // More info about this problem, see PR#9115. - colInfo.OriginDefaultValue, err = generateOriginDefaultValue(colInfo) + originDefVal, err := generateOriginDefaultValue(colInfo) if err != nil { return err } + return colInfo.SetOriginDefaultValue(originDefVal) } return nil } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index eacff4bc2d5a5..4a8b7a8a20328 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1118,6 +1118,18 @@ func (s *testIntegrationSuite5) TestBitDefaultValue(c *C) { tk.MustQuery("select c from t_bit").Check(testkit.Rows("\x19\xb9")) tk.MustExec("update t_bit set c = b'11100000000111'") tk.MustQuery("select c from t_bit").Check(testkit.Rows("\x38\x07")) + tk.MustExec("drop table t_bit") + + tk.MustExec("create table t_bit (a int)") + tk.MustExec("insert into t_bit value (1)") + tk.MustExec("alter table t_bit add column b bit(1) default b'0';") + tk.MustExec("alter table t_bit modify column b bit(1) default b'1';") + tk.MustQuery("select b from t_bit").Check(testkit.Rows("\x00")) + tk.MustExec("drop table t_bit") + + tk.MustExec("create table t_bit (a bit);") + tk.MustExec("insert into t_bit values (null);") + tk.MustQuery("select count(*) from t_bit where a is null;").Check(testkit.Rows("1")) tk.MustExec(`create table testalltypes1 ( field_1 bit default 1, @@ -2188,6 +2200,8 @@ func (s *testIntegrationSuite7) TestCreateExpressionIndexError(c *C) { tk.MustExec("drop table if exists t;") tk.MustGetErrCode("create table t (j json, key k (((j,j))))", errno.ErrFunctionalIndexRowValueIsNotAllowed) tk.MustExec("create table t (j json, key k ((j+1),(j+1)))") + + tk.MustGetErrCode("create table t1 (col1 int, index ((concat(''))));", errno.ErrWrongKeyColumnFunctionalIndex) } func (s *testIntegrationSuite7) TestAddExpressionIndexOnPartition(c *C) { diff --git a/ddl/ddl.go b/ddl/ddl.go index a764d51e0b755..096437fd40f60 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -110,6 +110,7 @@ type DDL interface { RepairTable(ctx sessionctx.Context, table *ast.TableName, createStmt *ast.CreateTableStmt) error CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error DropSequence(ctx sessionctx.Context, tableIdent ast.Ident, ifExists bool) (err error) + AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error // CreateSchemaWithInfo creates a database (schema) given its database info. // diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 43162467d9202..232ece09ec22e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -353,6 +353,25 @@ func ResolveCharsetCollation(charsetOpts ...ast.CharsetOpt) (string, string, err return chs, coll, nil } +// OverwriteCollationWithBinaryFlag is used to handle the case like +// CREATE TABLE t (a VARCHAR(255) BINARY) CHARSET utf8 COLLATE utf8_general_ci; +// The 'BINARY' sets the column collation to *_bin according to the table charset. +func OverwriteCollationWithBinaryFlag(colDef *ast.ColumnDef, chs, coll string) (newChs string, newColl string) { + ignoreBinFlag := colDef.Tp.Charset != "" && (colDef.Tp.Collate != "" || containsColumnOption(colDef, ast.ColumnOptionCollate)) + if ignoreBinFlag { + return chs, coll + } + needOverwriteBinColl := types.IsString(colDef.Tp.Tp) && mysql.HasBinaryFlag(colDef.Tp.Flag) + if needOverwriteBinColl { + newColl, err := charset.GetDefaultCollation(chs) + if err != nil { + return chs, coll + } + return chs, newColl + } + return chs, coll +} + func typesNeedCharset(tp byte) bool { switch tp { case mysql.TypeString, mysql.TypeVarchar, mysql.TypeVarString, @@ -412,6 +431,7 @@ func buildColumnAndConstraint( ast.CharsetOpt{Chs: chs, Col: coll}, ast.CharsetOpt{Chs: tblCharset, Col: tblCollate}, ) + chs, coll = OverwriteCollationWithBinaryFlag(colDef, chs, coll) if err != nil { return nil, nil, errors.Trace(err) } @@ -504,8 +524,12 @@ func isExplicitTimeStamp() bool { // processColumnFlags is used by columnDefToCol and processColumnOptions. It is intended to unify behaviors on `create/add` and `modify/change` statements. Check tidb#issue#19342. func processColumnFlags(col *table.Column) { - if col.FieldType.EvalType().IsStringKind() && col.Charset == charset.CharsetBin { - col.Flag |= mysql.BinaryFlag + if col.FieldType.EvalType().IsStringKind() { + if col.Charset == charset.CharsetBin { + col.Flag |= mysql.BinaryFlag + } else { + col.Flag &= ^mysql.BinaryFlag + } } if col.Tp == mysql.TypeBit { // For BIT field, it's charset is binary but does not have binary flag. @@ -2619,11 +2643,13 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model return nil, errors.Trace(err) } - col.OriginDefaultValue, err = generateOriginDefaultValue(col.ToInfo()) + originDefVal, err := generateOriginDefaultValue(col.ToInfo()) if err != nil { return nil, errors.Trace(err) } - return col, nil + + err = col.SetOriginDefaultValue(originDefVal) + return col, err } // AddColumn will add a new column to the table. @@ -3539,12 +3565,13 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or // a new version TiDB builds the DDL job that doesn't be set the column's offset and state, // and the old version TiDB is the DDL owner, it doesn't get offset and state from the store. Then it will encounter errors. // So here we set offset and state to support the rolling upgrade. - Offset: col.Offset, - State: col.State, - OriginDefaultValue: col.OriginDefaultValue, - FieldType: *specNewColumn.Tp, - Name: newColName, - Version: col.Version, + Offset: col.Offset, + State: col.State, + OriginDefaultValue: col.OriginDefaultValue, + OriginDefaultValueBit: col.OriginDefaultValueBit, + FieldType: *specNewColumn.Tp, + Name: newColName, + Version: col.Version, }) var chs, coll string @@ -3564,6 +3591,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or ast.CharsetOpt{Chs: t.Meta().Charset, Col: t.Meta().Collate}, ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, ) + chs, coll = OverwriteCollationWithBinaryFlag(specNewColumn, chs, coll) if err != nil { return nil, errors.Trace(err) } @@ -4543,7 +4571,7 @@ func buildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*as if err = checkDependedColExist(checkDependencies, existCols); err != nil { return nil, errors.Trace(err) } - if err = checkAutoIncrementRef("", colInfo.Dependences, tblInfo); err != nil { + if err = checkExpressionIndexAutoIncrement(indexName.O, colInfo.Dependences, tblInfo); err != nil { return nil, errors.Trace(err) } idxPart.Expr = nil @@ -5291,6 +5319,49 @@ func (d *ddl) CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStm return d.CreateTableWithInfo(ctx, ident.Schema, tbInfo, onExist, false /*tryRetainID*/) } +func (d *ddl) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error { + ident := ast.Ident{Name: stmt.Name.Name, Schema: stmt.Name.Schema} + is := d.GetInfoSchemaWithInterceptor(ctx) + // Check schema existence. + db, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + // Check table existence. + tbl, err := is.TableByName(ident.Schema, ident.Name) + if err != nil { + if stmt.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + if !tbl.Meta().IsSequence() { + return ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "SEQUENCE") + } + + // Validate the new sequence option value in old sequenceInfo. + oldSequenceInfo := tbl.Meta().Sequence + copySequenceInfo := *oldSequenceInfo + _, _, err = alterSequenceOptions(stmt.SeqOptions, ident, ©SequenceInfo) + if err != nil { + return err + } + + job := &model.Job{ + SchemaID: db.ID, + TableID: tbl.Meta().ID, + SchemaName: db.Name.L, + Type: model.ActionAlterSequence, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{ident, stmt.SeqOptions}, + } + + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + func (d *ddl) DropSequence(ctx sessionctx.Context, ti ast.Ident, ifExists bool) (err error) { schema, tbl, err := d.getSchemaAndTableByIdent(ctx, ti) if err != nil { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 38d547d67eb1c..dd79b95876c72 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -685,6 +685,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onAlterIndexVisibility(t, job) case model.ActionAlterTableAlterPartition: ver, err = onAlterTablePartition(t, job) + case model.ActionAlterSequence: + ver, err = onAlterSequence(t, job) default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/error.go b/ddl/error.go index c450c1f8e15e8..c75ccb1b6d487 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -64,6 +64,8 @@ var ( errOnlyOnRangeListPartition = terror.ClassDDL.New(mysql.ErrOnlyOnRangeListPartition, mysql.MySQLErrName[mysql.ErrOnlyOnRangeListPartition]) // errWrongKeyColumn is for table column cannot be indexed. errWrongKeyColumn = terror.ClassDDL.New(mysql.ErrWrongKeyColumn, mysql.MySQLErrName[mysql.ErrWrongKeyColumn]) + // errWrongKeyColumnFunctionalIndex is for expression cannot be indexed. + errWrongKeyColumnFunctionalIndex = terror.ClassDDL.New(mysql.ErrWrongKeyColumnFunctionalIndex, mysql.MySQLErrName[mysql.ErrWrongKeyColumnFunctionalIndex]) // errWrongFKOptionForGeneratedColumn is for wrong foreign key reference option on generated columns. errWrongFKOptionForGeneratedColumn = terror.ClassDDL.New(mysql.ErrWrongFKOptionForGeneratedColumn, mysql.MySQLErrName[mysql.ErrWrongFKOptionForGeneratedColumn]) // ErrUnsupportedOnGeneratedColumn is for unsupported actions on generated columns. @@ -81,6 +83,8 @@ var ( ErrInvalidDefaultValue = terror.ClassDDL.New(mysql.ErrInvalidDefault, mysql.MySQLErrName[mysql.ErrInvalidDefault]) // ErrGeneratedColumnRefAutoInc forbids to refer generated columns to auto-increment columns . ErrGeneratedColumnRefAutoInc = terror.ClassDDL.New(mysql.ErrGeneratedColumnRefAutoInc, mysql.MySQLErrName[mysql.ErrGeneratedColumnRefAutoInc]) + // ErrExpressionIndexCanNotRefer forbids to refer expression index to auto-increment column. + ErrExpressionIndexCanNotRefer = terror.ClassDDL.New(mysql.ErrFunctionalIndexRefAutoIncrement, mysql.MySQLErrName[mysql.ErrFunctionalIndexRefAutoIncrement]) // ErrUnsupportedAddPartition returns for does not support add partitions. ErrUnsupportedAddPartition = terror.ClassDDL.New(mysql.ErrUnsupportedDDLOperation, fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation], "add partitions")) // ErrUnsupportedCoalescePartition returns for does not support coalesce partitions. @@ -200,8 +204,10 @@ var ( ErrFunctionalIndexOnField = terror.ClassDDL.New(mysql.ErrFunctionalIndexOnField, mysql.MySQLErrName[mysql.ErrFunctionalIndexOnField]) // ErrInvalidAutoRandom returns when auto_random is used incorrectly. ErrInvalidAutoRandom = terror.ClassDDL.New(mysql.ErrInvalidAutoRandom, mysql.MySQLErrName[mysql.ErrInvalidAutoRandom]) - // ErrUnsupportedConstraintCheck returns when use ADD CONSTRAINT CHECK + // ErrUnsupportedConstraintCheck returns when use ADD CONSTRAINT CHECK. ErrUnsupportedConstraintCheck = terror.ClassDDL.New(mysql.ErrUnsupportedConstraintCheck, mysql.MySQLErrName[mysql.ErrUnsupportedConstraintCheck]) + // ErrDerivedMustHaveAlias returns when a sub select statement does not have a table alias. + ErrDerivedMustHaveAlias = terror.ClassDDL.New(mysql.ErrDerivedMustHaveAlias, mysql.MySQLErrName[mysql.ErrDerivedMustHaveAlias]) // ErrSequenceRunOut returns when the sequence has been run out. ErrSequenceRunOut = terror.ClassDDL.New(mysql.ErrSequenceRunOut, mysql.MySQLErrName[mysql.ErrSequenceRunOut]) diff --git a/ddl/generated_column.go b/ddl/generated_column.go index 63e25832a0709..d76b030a84434 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -331,3 +331,14 @@ func checkAutoIncrementRef(name string, dependencies map[string]struct{}, tbInfo } return nil } + +// checkExpressionIndexAutoIncrement checks if an generated column depends on an auto-increment column and raises an error if so. +func checkExpressionIndexAutoIncrement(name string, dependencies map[string]struct{}, tbInfo *model.TableInfo) error { + exists, autoIncrementColumn := infoschema.HasAutoIncrementColumn(tbInfo) + if exists { + if _, found := dependencies[autoIncrementColumn]; found { + return ErrExpressionIndexCanNotRefer.GenWithStackByArgs(name) + } + } + return nil +} diff --git a/ddl/index.go b/ddl/index.go index 26bcbcd411aa9..9b06678bfc575 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -130,6 +130,9 @@ func checkIndexPrefixLength(columns []*model.ColumnInfo, idxColumns []*model.Ind func checkIndexColumn(col *model.ColumnInfo, ic *ast.IndexPartSpecification) error { if col.Flen == 0 && (types.IsTypeChar(col.FieldType.Tp) || types.IsTypeVarchar(col.FieldType.Tp)) { + if col.GeneratedExprString != "" { + return errors.Trace(errWrongKeyColumnFunctionalIndex.GenWithStackByArgs(col.GeneratedExprString)) + } return errors.Trace(errWrongKeyColumn.GenWithStackByArgs(ic.Column.Name)) } diff --git a/ddl/sequence.go b/ddl/sequence.go index ce7b0bdcdd229..c7aa6c47f8507 100644 --- a/ddl/sequence.go +++ b/ddl/sequence.go @@ -15,6 +15,7 @@ package ddl import ( "math" + "reflect" "github.com/cznic/mathutil" "github.com/pingcap/errors" @@ -183,3 +184,110 @@ func buildSequenceInfo(stmt *ast.CreateSequenceStmt, ident ast.Ident) (*model.Se } return sequenceInfo, nil } + +func alterSequenceOptions(sequenceOptions []*ast.SequenceOption, ident ast.Ident, oldSequence *model.SequenceInfo) (bool, int64, error) { + var ( + restartFlag bool + restartWithFlag bool + restartValue int64 + ) + // Override the old sequence value with new option. + for _, op := range sequenceOptions { + switch op.Tp { + case ast.SequenceOptionIncrementBy: + oldSequence.Increment = op.IntValue + case ast.SequenceStartWith: + oldSequence.Start = op.IntValue + case ast.SequenceMinValue: + oldSequence.MinValue = op.IntValue + case ast.SequenceMaxValue: + oldSequence.MaxValue = op.IntValue + case ast.SequenceCache: + oldSequence.CacheValue = op.IntValue + case ast.SequenceNoCache: + oldSequence.Cache = false + case ast.SequenceCycle: + oldSequence.Cycle = true + case ast.SequenceNoCycle: + oldSequence.Cycle = false + case ast.SequenceRestart: + restartFlag = true + case ast.SequenceRestartWith: + restartWithFlag = true + restartValue = op.IntValue + } + } + if !validateSequenceOptions(oldSequence) { + return false, 0, ErrSequenceInvalidData.GenWithStackByArgs(ident.Schema.L, ident.Name.L) + } + if restartWithFlag { + return true, restartValue, nil + } + if restartFlag { + return true, oldSequence.Start, nil + } + return false, 0, nil +} + +func onAlterSequence(t *meta.Meta, job *model.Job) (ver int64, _ error) { + schemaID := job.SchemaID + var ( + sequenceOpts []*ast.SequenceOption + ident ast.Ident + ) + if err := job.DecodeArgs(&ident, &sequenceOpts); err != nil { + // Invalid arguments, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + // Get the old tableInfo. + tblInfo, err := checkTableExistAndCancelNonExistJob(t, job, schemaID) + if err != nil { + return ver, errors.Trace(err) + } + + // Substitute the sequence info. + copySequenceInfo := *tblInfo.Sequence + restart, restartValue, err := alterSequenceOptions(sequenceOpts, ident, ©SequenceInfo) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + shouldUpdateVer := !reflect.DeepEqual(*tblInfo.Sequence, copySequenceInfo) || restart + tblInfo.Sequence = ©SequenceInfo + + // Restart the sequence value. + // Notice: during the alter sequence process, if there is some dml continually consumes sequence (nextval/setval), + // the below cases will occur: + // Since the table schema haven't been refreshed in local/other node, dml will still use old definition of sequence + // to allocate sequence ids. Once the restart value is updated to kv here, the allocated ids in the upper layer won't + // guarantee to be consecutive and monotonous. + if restart { + err := restartSequenceValue(t, schemaID, tblInfo, restartValue) + if err != nil { + return ver, errors.Trace(err) + } + } + + // Store the sequence info into kv. + ver, err = updateVersionAndTableInfo(t, job, tblInfo, shouldUpdateVer) + if err != nil { + return ver, errors.Trace(err) + } + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return ver, nil +} + +// Like setval does, restart sequence value won't affect current the step frequency. It will look backward for +// the first valid sequence valid rather than return the restart value directly. +func restartSequenceValue(t *meta.Meta, dbID int64, tblInfo *model.TableInfo, seqValue int64) error { + var sequenceBase int64 + if tblInfo.Sequence.Increment >= 0 { + sequenceBase = seqValue - 1 + } else { + sequenceBase = seqValue + 1 + } + return t.RestartSequenceValue(dbID, tblInfo, sequenceBase) +} diff --git a/ddl/sequence_test.go b/ddl/sequence_test.go index 31b41c2bd7d00..f46da112888df 100644 --- a/ddl/sequence_test.go +++ b/ddl/sequence_test.go @@ -74,6 +74,7 @@ func (s *testSequenceSuite) TestCreateSequence(c *C) { c.Assert(sequenceTable.Meta().Sequence.Cycle, Equals, false) // Test create privilege. + tk.MustExec("drop user if exists myuser@localhost") tk.MustExec("create user myuser@localhost") tk1 := testkit.NewTestKit(c, s.store) @@ -989,3 +990,87 @@ func (s *testSequenceSuite) TestSequenceCacheShouldNotBeNegative(c *C) { tk.MustExec("create sequence seq cache 1") } + +func (s *testSequenceSuite) TestAlterSequence(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("drop sequence if exists seq") + tk.MustExec("create sequence seq") + tk.MustExec("alter sequence seq increment by 2 start with 2") + tk.MustQuery("show create sequence seq").Check(testkit.Rows("seq CREATE SEQUENCE `seq` " + + "start with 2 minvalue 1 maxvalue 9223372036854775806 increment by 2 cache 1000 nocycle ENGINE=InnoDB")) + + tk.MustExec("drop sequence if exists seq") + tk.MustExec("create sequence seq") + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("2")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3")) + // Alter sequence will invalidate the sequence cache in memory. + tk.MustExec("alter sequence seq increment by 2") + tk.MustQuery("show create sequence seq").Check(testkit.Rows("seq CREATE SEQUENCE `seq` " + + "start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 2 cache 1000 nocycle ENGINE=InnoDB")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1001")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1003")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1005")) + + // Restart with value will reset the sequence value in kv. + tk.MustExec("alter sequence seq restart with 9") + // Like setval does, the sequence current value change won't affect the increment frequency. + // By now the step frequency is: 1, 3, 5, 7, 9, 11, 13, 15... + // After restart with 9, the current value rebased to 8, the next valid value will be 9, coincidentally equal to what we restarted. + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("9")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("11")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("13")) + + tk.MustExec("alter sequence seq restart with 10") + // After restart with 10, the current value rebased to 9, the next valid value will be 11, rather than what we restart. + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("11")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("13")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("15")) + + // Restart will reset the sequence value to start value by default. + tk.MustExec("alter sequence seq restart") + // After restart, the base will be pointed to 0, the first value will be 1 here, then plus the increment 2, the second value will be 3. + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("5")) + + tk.MustExec("drop sequence if exists seq") + tk.MustExec("create sequence seq increment by 3") + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("4")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("7")) + + tk.MustExec("alter sequence seq increment by 4") + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3001")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3005")) + tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3009")) + tk.MustExec("drop sequence if exists seq") +} + +func (s *testSequenceSuite) TestAlterSequencePrivilege(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop sequence if exists my_seq") + tk.MustExec("create sequence my_seq") + + // Test create privilege. + tk.MustExec("drop user if exists myuser@localhost") + tk.MustExec("create user myuser@localhost") + + tk1 := testkit.NewTestKit(c, s.store) + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + c.Assert(se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil), IsTrue) + tk1.Se = se + + // grant the myuser the access to database test. + tk.MustExec("grant select on test.* to 'myuser'@'localhost'") + + tk1.MustExec("use test") + _, err = tk1.Exec("alter sequence my_seq increment = 2") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[planner:1142]ALTER command denied to user 'myuser'@'localhost' for table 'my_seq'") + tk.MustExec("drop sequence if exists my_seq") +} diff --git a/domain/domain.go b/domain/domain.go index a2557f71d3f81..1c674ac5fe38a 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1033,8 +1033,13 @@ func (do *Domain) StatsHandle() *handle.Handle { } // CreateStatsHandle is used only for test. -func (do *Domain) CreateStatsHandle(ctx sessionctx.Context) { - atomic.StorePointer(&do.statsHandle, unsafe.Pointer(handle.NewHandle(ctx, do.statsLease))) +func (do *Domain) CreateStatsHandle(ctx sessionctx.Context) error { + h, err := handle.NewHandle(ctx, do.statsLease) + if err != nil { + return err + } + atomic.StorePointer(&do.statsHandle, unsafe.Pointer(h)) + return nil } // StatsUpdating checks if the stats worker is updating. @@ -1059,7 +1064,10 @@ var RunAutoAnalyze = true // It should be called only once in BootstrapSession. func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error { ctx.GetSessionVars().InRestrictedSQL = true - statsHandle := handle.NewHandle(ctx, do.statsLease) + statsHandle, err := handle.NewHandle(ctx, do.statsLease) + if err != nil { + return err + } atomic.StorePointer(&do.statsHandle, unsafe.Pointer(statsHandle)) do.ddl.RegisterEventCh(statsHandle.DDLEventCh()) // Negative stats lease indicates that it is in test, it does not need update. @@ -1120,6 +1128,10 @@ func (do *Domain) loadStatsWorker() { for { select { case <-loadTicker.C: + err = statsHandle.RefreshVars() + if err != nil { + logutil.BgLogger().Debug("refresh variables failed", zap.Error(err)) + } err = statsHandle.Update(do.InfoSchema()) if err != nil { logutil.BgLogger().Debug("update stats info failed", zap.Error(err)) diff --git a/errno/errname.go b/errno/errname.go index 0a3e0e858a2f3..9aa2b9a857ae6 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -868,25 +868,25 @@ var MySQLErrName = map[uint16]string{ ErrRoleNotGranted: "%s is is not granted to %s", ErrMaxExecTimeExceeded: "Query execution was interrupted, max_execution_time exceeded.", ErrLockAcquireFailAndNoWaitSet: "Statement aborted because lock(s) could not be acquired immediately and NOWAIT is set.", - ErrDataTruncatedFunctionalIndex: "Data truncated for functional index '%s' at row %d", - ErrDataOutOfRangeFunctionalIndex: "Value is out of range for functional index '%s' at row %d", - ErrFunctionalIndexOnJSONOrGeometryFunction: "Cannot create a functional index on a function that returns a JSON or GEOMETRY value", - ErrFunctionalIndexRefAutoIncrement: "Functional index '%s' cannot refer to an auto-increment column", - ErrCannotDropColumnFunctionalIndex: "Cannot drop column '%s' because it is used by a functional index. In order to drop the column, you must remove the functional index", - ErrFunctionalIndexPrimaryKey: "The primary key cannot be a functional index", - ErrFunctionalIndexOnLob: "Cannot create a functional index on an expression that returns a BLOB or TEXT. Please consider using CAST", - ErrFunctionalIndexFunctionIsNotAllowed: "Expression of functional index '%s' contains a disallowed function", - ErrFulltextFunctionalIndex: "Fulltext functional index is not supported", - ErrSpatialFunctionalIndex: "Spatial functional index is not supported", + ErrDataTruncatedFunctionalIndex: "Data truncated for expression index '%s' at row %d", + ErrDataOutOfRangeFunctionalIndex: "Value is out of range for expression index '%s' at row %d", + ErrFunctionalIndexOnJSONOrGeometryFunction: "Cannot create an expression index on a function that returns a JSON or GEOMETRY value", + ErrFunctionalIndexRefAutoIncrement: "Expression index '%s' cannot refer to an auto-increment column", + ErrCannotDropColumnFunctionalIndex: "Cannot drop column '%s' because it is used by an expression index. In order to drop the column, you must remove the expression index", + ErrFunctionalIndexPrimaryKey: "The primary key cannot be an expression index", + ErrFunctionalIndexOnLob: "Cannot create an expression index on an expression that returns a BLOB or TEXT. Please consider using CAST", + ErrFunctionalIndexFunctionIsNotAllowed: "Expression of expression index '%s' contains a disallowed function", + ErrFulltextFunctionalIndex: "Fulltext expression index is not supported", + ErrSpatialFunctionalIndex: "Spatial expression index is not supported", ErrWrongKeyColumnFunctionalIndex: "The used storage engine cannot index the expression '%s'", - ErrFunctionalIndexOnField: "Functional index on a column is not supported. Consider using a regular index instead", + ErrFunctionalIndexOnField: "Expression index on a column is not supported. Consider using a regular index instead", ErrFKIncompatibleColumns: "Referencing column '%s' in foreign key constraint '%s' are incompatible", - ErrFunctionalIndexRowValueIsNotAllowed: "Expression of functional index '%s' cannot refer to a row value", - ErrDependentByFunctionalIndex: "Column '%s' has a functional index dependency and cannot be dropped or renamed", - ErrInvalidJSONValueForFuncIndex: "Invalid JSON value for CAST for functional index '%s'", - ErrJSONValueOutOfRangeForFuncIndex: "Out of range JSON value for CAST for functional index '%s'", - ErrFunctionalIndexDataIsTooLong: "Data too long for functional index '%s'", - ErrFunctionalIndexNotApplicable: "Cannot use functional index '%s' due to type or collation conversion", + ErrFunctionalIndexRowValueIsNotAllowed: "Expression of expression index '%s' cannot refer to a row value", + ErrDependentByFunctionalIndex: "Column '%s' has an expression index dependency and cannot be dropped or renamed", + ErrInvalidJSONValueForFuncIndex: "Invalid JSON value for CAST for expression index '%s'", + ErrJSONValueOutOfRangeForFuncIndex: "Out of range JSON value for CAST for expression index '%s'", + ErrFunctionalIndexDataIsTooLong: "Data too long for expression index '%s'", + ErrFunctionalIndexNotApplicable: "Cannot use expression index '%s' due to type or collation conversion", ErrUnsupportedConstraintCheck: "%s is not supported", // MariaDB errors. ErrOnlyOneDefaultPartionAllowed: "Only one DEFAULT partition allowed", diff --git a/executor/adapter.go b/executor/adapter.go index 823e196e7db44..2c8ff3c1a6a4d 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -387,19 +387,22 @@ func (a *ExecStmt) handleNoDelay(ctx context.Context, e Executor, isPessimistic }() toCheck := e + isExplainAnalyze := false if explain, ok := e.(*ExplainExec); ok { - if explain.analyzeExec != nil { - toCheck = explain.analyzeExec + if analyze := explain.getAnalyzeExecToExecutedNoDelay(); analyze != nil { + toCheck = analyze + isExplainAnalyze = true } } // If the executor doesn't return any result to the client, we execute it without delay. if toCheck.Schema().Len() == 0 { + handled = !isExplainAnalyze if isPessimistic { - return true, nil, a.handlePessimisticDML(ctx, e) + return handled, nil, a.handlePessimisticDML(ctx, toCheck) } - r, err := a.handleNoDelayExecutor(ctx, e) - return true, r, err + r, err := a.handleNoDelayExecutor(ctx, toCheck) + return handled, r, err } else if proj, ok := toCheck.(*ProjectionExec); ok && proj.calculateNoDelay { // Currently this is only for the "DO" statement. Take "DO 1, @a=2;" as an example: // the Projection has two expressions and two columns in the schema, but we should @@ -589,6 +592,15 @@ func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error { if !txn.Valid() { return errors.Trace(kv.ErrInvalidTxn) } + + // The Oracle serializable isolation is actually SI in pessimistic mode. + // Do not update ForUpdateTS when the user is using the Serializable isolation level. + // It can be used temporarily on the few occasions when an Oracle-like isolation level is needed. + // Support for this does not mean that TiDB supports serializable isolation of MySQL. + // tidb_skip_isolation_level_check should still be disabled by default. + if seCtx.GetSessionVars().IsIsolation(ast.Serializable) { + return nil + } if newForUpdateTS == 0 { version, err := seCtx.GetStore().CurrentVersion() if err != nil { @@ -603,7 +615,11 @@ func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error { // handlePessimisticLockError updates TS and rebuild executor if the err is write conflict. func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, err error) (Executor, error) { - txnCtx := a.Ctx.GetSessionVars().TxnCtx + sessVars := a.Ctx.GetSessionVars() + if err != nil && sessVars.IsIsolation(ast.Serializable) { + return nil, err + } + txnCtx := sessVars.TxnCtx var newForUpdateTS uint64 if deadlock, ok := errors.Cause(err).(*tikv.ErrDeadlock); ok { if !deadlock.IsRetryable { diff --git a/executor/aggfuncs/aggfunc_test.go b/executor/aggfuncs/aggfunc_test.go index d7dc3703dd4a9..d04ee889e4524 100644 --- a/executor/aggfuncs/aggfunc_test.go +++ b/executor/aggfuncs/aggfunc_test.go @@ -45,6 +45,11 @@ import ( "github.com/pingcap/tidb/util/set" ) +const ( + // separator argument for group_concat() test cases + separator = " " +) + var _ = Suite(&testSuite{}) func TestT(t *testing.T) { @@ -264,9 +269,9 @@ func rowMemDeltaGens(srcChk *chunk.Chunk, dataType *types.FieldType) (memDeltas return memDeltas, nil } -type multiArgsUpdateMemDeltaGens func(*chunk.Chunk, []*types.FieldType) (memDeltas []int64, err error) +type multiArgsUpdateMemDeltaGens func(*chunk.Chunk, []*types.FieldType, []*util.ByItems) (memDeltas []int64, err error) -func defaultMultiArgsMemDeltaGens(srcChk *chunk.Chunk, dataTypes []*types.FieldType) (memDeltas []int64, err error) { +func defaultMultiArgsMemDeltaGens(srcChk *chunk.Chunk, dataTypes []*types.FieldType, byItems []*util.ByItems) (memDeltas []int64, err error) { memDeltas = make([]int64, 0) m := make(map[string]bool) for i := 0; i < srcChk.NumRows(); i++ { @@ -342,14 +347,16 @@ type multiArgsAggMemTest struct { multiArgsAggTest multiArgsAggTest allocMemDelta int64 multiArgsUpdateMemDeltaGens multiArgsUpdateMemDeltaGens + isDistinct bool } -func buildMultiArgsAggMemTester(funcName string, tps []byte, rt byte, numRows int, allocMemDelta int64, updateMemDeltaGens multiArgsUpdateMemDeltaGens, results ...interface{}) multiArgsAggMemTest { - multiArgsAggTest := buildMultiArgsAggTester(funcName, tps, rt, numRows, results...) +func buildMultiArgsAggMemTester(funcName string, tps []byte, rt byte, numRows int, allocMemDelta int64, updateMemDeltaGens multiArgsUpdateMemDeltaGens, isDistinct bool) multiArgsAggMemTest { + multiArgsAggTest := buildMultiArgsAggTester(funcName, tps, rt, numRows) pt := multiArgsAggMemTest{ multiArgsAggTest: multiArgsAggTest, allocMemDelta: allocMemDelta, multiArgsUpdateMemDeltaGens: updateMemDeltaGens, + isDistinct: isDistinct, } return pt } @@ -360,7 +367,7 @@ func (s *testSuite) testMergePartialResult(c *C, p aggTest) { args := []expression.Expression{&expression.Column{RetType: p.dataType, Index: 0}} if p.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) c.Assert(err, IsNil) @@ -589,7 +596,7 @@ func (s *testSuite) testAggFunc(c *C, p aggTest) { args := []expression.Expression{&expression.Column{RetType: p.dataType, Index: 0}} if p.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } if p.funcName == ast.AggFuncApproxPercentile { args = append(args, &expression.Constant{Value: types.NewIntDatum(50), RetType: types.NewFieldType(mysql.TypeLong)}) @@ -670,7 +677,7 @@ func (s *testSuite) testAggMemFunc(c *C, p aggMemTest) { args := []expression.Expression{&expression.Column{RetType: p.aggTest.dataType, Index: 0}} if p.aggTest.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } desc, err := aggregation.NewAggFuncDesc(s.ctx, p.aggTest.funcName, args, p.isDistinct) c.Assert(err, IsNil) @@ -703,7 +710,7 @@ func (s *testSuite) testMultiArgsAggFunc(c *C, p multiArgsAggTest) { args[k] = &expression.Column{RetType: p.dataTypes[k], Index: k} } if p.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) @@ -785,10 +792,10 @@ func (s *testSuite) testMultiArgsAggMemFunc(c *C, p multiArgsAggMemTest) { args[k] = &expression.Column{RetType: p.multiArgsAggTest.dataTypes[k], Index: k} } if p.multiArgsAggTest.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } - desc, err := aggregation.NewAggFuncDesc(s.ctx, p.multiArgsAggTest.funcName, args, false) + desc, err := aggregation.NewAggFuncDesc(s.ctx, p.multiArgsAggTest.funcName, args, p.isDistinct) c.Assert(err, IsNil) if p.multiArgsAggTest.orderBy { desc.OrderByItems = []*util.ByItems{ @@ -799,8 +806,7 @@ func (s *testSuite) testMultiArgsAggMemFunc(c *C, p multiArgsAggMemTest) { finalPr, memDelta := finalFunc.AllocPartialResult() c.Assert(memDelta, Equals, p.allocMemDelta) - resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{desc.RetTp}, 1) - updateMemDeltas, err := p.multiArgsUpdateMemDeltaGens(srcChk, p.multiArgsAggTest.dataTypes) + updateMemDeltas, err := p.multiArgsUpdateMemDeltaGens(srcChk, p.multiArgsAggTest.dataTypes, desc.OrderByItems) c.Assert(err, IsNil) iter := chunk.NewIterator4Chunk(srcChk) i := 0 @@ -809,30 +815,6 @@ func (s *testSuite) testMultiArgsAggMemFunc(c *C, p multiArgsAggMemTest) { c.Assert(memDelta, Equals, updateMemDeltas[i]) i++ } - - // test the agg func with distinct - desc, err = aggregation.NewAggFuncDesc(s.ctx, p.multiArgsAggTest.funcName, args, true) - c.Assert(err, IsNil) - if p.multiArgsAggTest.orderBy { - desc.OrderByItems = []*util.ByItems{ - {Expr: args[0], Desc: true}, - } - } - finalFunc = aggfuncs.Build(s.ctx, desc, 0) - finalPr, memDelta = finalFunc.AllocPartialResult() - c.Assert(memDelta, Equals, p.allocMemDelta) - - resultChk.Reset() - srcChk = p.multiArgsAggTest.genSrcChk() - updateMemDeltas, err = p.multiArgsUpdateMemDeltaGens(srcChk, p.multiArgsAggTest.dataTypes) - c.Assert(err, IsNil) - iter = chunk.NewIterator4Chunk(srcChk) - i = 0 - for row := iter.Begin(); row != iter.End(); row = iter.Next() { - memDelta, _ := finalFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, finalPr) - c.Assert(memDelta, Equals, updateMemDeltas[i]) - i++ - } } func (s *testSuite) benchmarkAggFunc(b *testing.B, p aggTest) { @@ -845,7 +827,7 @@ func (s *testSuite) benchmarkAggFunc(b *testing.B, p aggTest) { args := []expression.Expression{&expression.Column{RetType: p.dataType, Index: 0}} if p.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) if err != nil { @@ -898,7 +880,7 @@ func (s *testSuite) benchmarkMultiArgsAggFunc(b *testing.B, p multiArgsAggTest) args[k] = &expression.Column{RetType: p.dataTypes[k], Index: k} } if p.funcName == ast.AggFuncGroupConcat { - args = append(args, &expression.Constant{Value: types.NewStringDatum(" "), RetType: types.NewFieldType(mysql.TypeString)}) + args = append(args, &expression.Constant{Value: types.NewStringDatum(separator), RetType: types.NewFieldType(mysql.TypeString)}) } desc, err := aggregation.NewAggFuncDesc(s.ctx, p.funcName, args, false) diff --git a/executor/aggfuncs/func_count_distinct.go b/executor/aggfuncs/func_count_distinct.go index 29f8338a9c19c..0dd974b390283 100644 --- a/executor/aggfuncs/func_count_distinct.go +++ b/executor/aggfuncs/func_count_distinct.go @@ -428,7 +428,8 @@ func appendDecimal(encodedBytes []byte, val *types.MyDecimal) ([]byte, error) { return encodedBytes, err } -func writeTime(buf []byte, t types.Time) { +// WriteTime writes `t` into `buf`. +func WriteTime(buf []byte, t types.Time) { binary.BigEndian.PutUint16(buf, uint16(t.Year())) buf[2] = uint8(t.Month()) buf[3] = uint8(t.Day()) @@ -438,10 +439,12 @@ func writeTime(buf []byte, t types.Time) { binary.BigEndian.PutUint32(buf[8:], uint32(t.Microsecond())) buf[12] = t.Type() buf[13] = uint8(t.Fsp()) + + buf[7], buf[14], buf[15] = uint8(0), uint8(0), uint8(0) } func appendTime(encodedBytes, buf []byte, val types.Time) []byte { - writeTime(buf, val) + WriteTime(buf, val) buf = buf[:16] encodedBytes = append(encodedBytes, buf...) return encodedBytes diff --git a/executor/aggfuncs/func_count_test.go b/executor/aggfuncs/func_count_test.go index eb6bf296dd743..3b6898a8cdedb 100644 --- a/executor/aggfuncs/func_count_test.go +++ b/executor/aggfuncs/func_count_test.go @@ -22,6 +22,8 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/executor/aggfuncs" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/types" ) func genApproxDistinctMergePartialResult(begin, end uint64) string { @@ -143,6 +145,20 @@ func (s *testSuite) TestMemCount(c *C) { } } +func (s *testSuite) TestWriteTime(c *C) { + t, err := types.ParseDate(&(stmtctx.StatementContext{}), "2020-11-11") + c.Assert(err, IsNil) + + buf := make([]byte, 16) + for i := range buf { + buf[i] = uint8(255) + } + aggfuncs.WriteTime(buf, t) + for i := range buf { + c.Assert(buf[i] == uint8(255), IsFalse) + } +} + func BenchmarkCount(b *testing.B) { s := testSuite{} s.SetUpSuite(nil) diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index 1e4b24bab45ca..0e2be59856eb2 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -18,6 +18,7 @@ import ( "container/heap" "sort" "sync/atomic" + "unsafe" "github.com/pingcap/parser/terror" mysql "github.com/pingcap/tidb/errno" @@ -31,6 +32,17 @@ import ( "github.com/pingcap/tidb/util/set" ) +const ( + // DefPartialResult4GroupConcatSize is the size of partialResult4GroupConcat + DefPartialResult4GroupConcatSize = int64(unsafe.Sizeof(partialResult4GroupConcat{})) + // DefPartialResult4GroupConcatDistinctSize is the size of partialResult4GroupConcatDistinct + DefPartialResult4GroupConcatDistinctSize = int64(unsafe.Sizeof(partialResult4GroupConcatDistinct{})) + // DefPartialResult4GroupConcatOrderSize is the size of partialResult4GroupConcatOrder + DefPartialResult4GroupConcatOrderSize = int64(unsafe.Sizeof(partialResult4GroupConcatOrder{})) + // DefPartialResult4GroupConcatOrderDistinctSize is the size of partialResult4GroupConcatOrderDistinct + DefPartialResult4GroupConcatOrderDistinctSize = int64(unsafe.Sizeof(partialResult4GroupConcatOrderDistinct{})) +) + type baseGroupConcat4String struct { baseAggFunc byItems []*util.ByItems @@ -87,7 +99,7 @@ type groupConcat struct { func (e *groupConcat) AllocPartialResult() (pr PartialResult, memDelta int64) { p := new(partialResult4GroupConcat) p.valsBuf = &bytes.Buffer{} - return PartialResult(p), 0 + return PartialResult(p), DefPartialResult4GroupConcatSize } func (e *groupConcat) ResetPartialResult(pr PartialResult) { @@ -103,7 +115,7 @@ func (e *groupConcat) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ for _, arg := range e.args { v, isNull, err = arg.EvalString(sctx, row) if err != nil { - return 0, err + return memDelta, err } if isNull { break @@ -113,17 +125,21 @@ func (e *groupConcat) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ if isNull { continue } + var oldMem int if p.buffer == nil { p.buffer = &bytes.Buffer{} } else { + oldMem = p.buffer.Cap() p.buffer.WriteString(e.sep) } p.buffer.WriteString(p.valsBuf.String()) + newMem := p.buffer.Cap() + memDelta += int64(newMem - oldMem) } if p.buffer != nil { - return 0, e.truncatePartialResultIfNeed(sctx, p.buffer) + return memDelta, e.truncatePartialResultIfNeed(sctx, p.buffer) } - return 0, nil + return memDelta, nil } func (e *groupConcat) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) (memDelta int64, err error) { @@ -164,7 +180,7 @@ func (e *groupConcatDistinct) AllocPartialResult() (pr PartialResult, memDelta i p := new(partialResult4GroupConcatDistinct) p.valsBuf = &bytes.Buffer{} p.valSet = set.NewStringSet() - return PartialResult(p), 0 + return PartialResult(p), DefPartialResult4GroupConcatDistinctSize } func (e *groupConcatDistinct) ResetPartialResult(pr PartialResult) { @@ -181,7 +197,7 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI for _, arg := range e.args { v, isNull, err = arg.EvalString(sctx, row) if err != nil { - return 0, err + return memDelta, err } if isNull { break @@ -197,19 +213,24 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI continue } p.valSet.Insert(joinedVal) + memDelta += int64(len(joinedVal)) + var oldMem int // write separator if p.buffer == nil { p.buffer = &bytes.Buffer{} } else { + oldMem = p.buffer.Cap() p.buffer.WriteString(e.sep) } // write values p.buffer.WriteString(p.valsBuf.String()) + newMem := p.buffer.Cap() + memDelta += int64(newMem - oldMem) } if p.buffer != nil { - return 0, e.truncatePartialResultIfNeed(sctx, p.buffer) + return memDelta, e.truncatePartialResultIfNeed(sctx, p.buffer) } - return 0, nil + return memDelta, nil } // SetTruncated will be called in `executorBuilder#buildHashAgg` with duck-type. @@ -278,14 +299,19 @@ func (h *topNRows) Pop() interface{} { return x } -func (h *topNRows) tryToAdd(row sortRow) (truncated bool) { +func (h *topNRows) tryToAdd(row sortRow) (truncated bool, sortRowMemSize int64) { + oldSize := h.currSize h.currSize += uint64(row.buffer.Len()) if len(h.rows) > 0 { h.currSize += h.sepSize } heap.Push(h, row) + for _, dt := range row.byItems { + sortRowMemSize += GetDatumMemSize(dt) + } if h.currSize <= h.limitSize { - return false + sortRowMemSize += int64(h.currSize - oldSize) + return false, sortRowMemSize } for h.currSize > h.limitSize { @@ -295,10 +321,14 @@ func (h *topNRows) tryToAdd(row sortRow) (truncated bool) { h.rows[0].buffer.Truncate(h.rows[0].buffer.Len() - int(debt)) } else { h.currSize -= uint64(h.rows[0].buffer.Len()) + h.sepSize + for _, dt := range h.rows[0].byItems { + sortRowMemSize -= GetDatumMemSize(dt) + } heap.Pop(h) } } - return true + sortRowMemSize += int64(h.currSize - oldSize) + return true, sortRowMemSize } func (h *topNRows) reset() { @@ -355,7 +385,7 @@ func (e *groupConcatOrder) AllocPartialResult() (pr PartialResult, memDelta int6 sepSize: uint64(len(e.sep)), }, } - return PartialResult(p), 0 + return PartialResult(p), DefPartialResult4GroupConcatOrderSize } func (e *groupConcatOrder) ResetPartialResult(pr PartialResult) { @@ -372,7 +402,7 @@ func (e *groupConcatOrder) UpdatePartialResult(sctx sessionctx.Context, rowsInGr for _, arg := range e.args { v, isNull, err = arg.EvalString(sctx, row) if err != nil { - return 0, err + return memDelta, err } if isNull { break @@ -389,21 +419,22 @@ func (e *groupConcatOrder) UpdatePartialResult(sctx sessionctx.Context, rowsInGr for _, byItem := range e.byItems { d, err := byItem.Expr.Eval(row) if err != nil { - return 0, err + return memDelta, err } sortRow.byItems = append(sortRow.byItems, d.Clone()) } - truncated := p.topN.tryToAdd(sortRow) + truncated, sortRowMemSize := p.topN.tryToAdd(sortRow) + memDelta += sortRowMemSize if p.topN.err != nil { - return 0, p.topN.err + return memDelta, p.topN.err } if truncated { if err := e.handleTruncateError(sctx); err != nil { - return 0, err + return memDelta, err } } } - return 0, nil + return memDelta, nil } func (e *groupConcatOrder) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) (memDelta int64, err error) { @@ -456,7 +487,7 @@ func (e *groupConcatDistinctOrder) AllocPartialResult() (pr PartialResult, memDe }, valSet: set.NewStringSet(), } - return PartialResult(p), 0 + return PartialResult(p), DefPartialResult4GroupConcatOrderDistinctSize } func (e *groupConcatDistinctOrder) ResetPartialResult(pr PartialResult) { @@ -475,7 +506,7 @@ func (e *groupConcatDistinctOrder) UpdatePartialResult(sctx sessionctx.Context, for _, arg := range e.args { v, isNull, err = arg.EvalString(sctx, row) if err != nil { - return 0, err + return memDelta, err } if isNull { break @@ -498,21 +529,23 @@ func (e *groupConcatDistinctOrder) UpdatePartialResult(sctx sessionctx.Context, for _, byItem := range e.byItems { d, err := byItem.Expr.Eval(row) if err != nil { - return 0, err + return memDelta, err } sortRow.byItems = append(sortRow.byItems, d.Clone()) } - truncated := p.topN.tryToAdd(sortRow) + truncated, sortRowMemSize := p.topN.tryToAdd(sortRow) + memDelta += sortRowMemSize + memDelta += int64(len(joinedVal)) if p.topN.err != nil { - return 0, p.topN.err + return memDelta, p.topN.err } if truncated { if err := e.handleTruncateError(sctx); err != nil { - return 0, err + return memDelta, err } } } - return 0, nil + return memDelta, nil } func (e *groupConcatDistinctOrder) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) (memDelta int64, err error) { @@ -520,3 +553,14 @@ func (e *groupConcatDistinctOrder) MergePartialResult(sctx sessionctx.Context, s // So MergePartialResult will not be called. return 0, terror.ClassOptimizer.New(mysql.ErrInternal, mysql.MySQLErrName[mysql.ErrInternal]).GenWithStack("groupConcatDistinctOrder.MergePartialResult should not be called") } + +// GetDatumMemSize calculates the memory size of each types.Datum in sortRow.byItems. +// types.Datum memory size = variable type's memory size + variable value's memory size. +func GetDatumMemSize(d *types.Datum) int64 { + var datumMemSize int64 + datumMemSize += int64(unsafe.Sizeof(*d)) + datumMemSize += int64(len(d.Collation())) + datumMemSize += int64(len(d.GetBytes())) + datumMemSize += getValMemDelta(d.GetInterface()) - DefInterfaceSize + return datumMemSize +} diff --git a/executor/aggfuncs/func_group_concat_test.go b/executor/aggfuncs/func_group_concat_test.go index 7e68e93cfed3b..76051d99b09ad 100644 --- a/executor/aggfuncs/func_group_concat_test.go +++ b/executor/aggfuncs/func_group_concat_test.go @@ -14,13 +14,20 @@ package aggfuncs_test import ( + "bytes" "fmt" . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/executor/aggfuncs" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/set" ) func (s *testSuite) TestMergePartialResult4GroupConcat(c *C) { @@ -45,3 +52,148 @@ func (s *testSuite) TestGroupConcat(c *C) { s.testMultiArgsAggFunc(c, test2) } } + +func (s *testSuite) TestMemGroupConcat(c *C) { + multiArgsTest1 := buildMultiArgsAggMemTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, + aggfuncs.DefPartialResult4GroupConcatSize, groupConcatMultiArgsUpdateMemDeltaGens, false) + multiArgsTest2 := buildMultiArgsAggMemTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, + aggfuncs.DefPartialResult4GroupConcatDistinctSize, groupConcatDistinctMultiArgsUpdateMemDeltaGens, true) + + multiArgsTest3 := buildMultiArgsAggMemTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, + aggfuncs.DefPartialResult4GroupConcatOrderSize, groupConcatOrderMultiArgsUpdateMemDeltaGens, false) + multiArgsTest3.multiArgsAggTest.orderBy = true + multiArgsTest4 := buildMultiArgsAggMemTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, + aggfuncs.DefPartialResult4GroupConcatOrderDistinctSize, groupConcatDistinctOrderMultiArgsUpdateMemDeltaGens, true) + multiArgsTest4.multiArgsAggTest.orderBy = true + + multiArgsTests := []multiArgsAggMemTest{multiArgsTest1, multiArgsTest2, multiArgsTest3, multiArgsTest4} + for _, test := range multiArgsTests { + s.testMultiArgsAggMemFunc(c, test) + } +} + +func groupConcatMultiArgsUpdateMemDeltaGens(srcChk *chunk.Chunk, dataType []*types.FieldType, byItems []*util.ByItems) (memDeltas []int64, err error) { + memDeltas = make([]int64, 0) + buffer := new(bytes.Buffer) + for i := 0; i < srcChk.NumRows(); i++ { + row := srcChk.GetRow(i) + if row.IsNull(0) { + memDeltas = append(memDeltas, int64(0)) + continue + } + oldMemSize := buffer.Cap() + if i != 0 { + buffer.WriteString(separator) + } + for j := 0; j < len(dataType); j++ { + curVal := row.GetString(j) + buffer.WriteString(curVal) + } + memDelta := int64(buffer.Cap() - oldMemSize) + memDeltas = append(memDeltas, memDelta) + } + return memDeltas, nil +} + +func groupConcatOrderMultiArgsUpdateMemDeltaGens(srcChk *chunk.Chunk, dataType []*types.FieldType, byItems []*util.ByItems) (memDeltas []int64, err error) { + memDeltas = make([]int64, 0) + buffer := new(bytes.Buffer) + for i := 0; i < srcChk.NumRows(); i++ { + row := srcChk.GetRow(i) + if row.IsNull(0) { + memDeltas = append(memDeltas, int64(0)) + continue + } + oldMemSize := buffer.Len() + if i != 0 { + buffer.WriteString(separator) + } + for j := 0; j < len(dataType); j++ { + curVal := row.GetString(j) + buffer.WriteString(curVal) + } + memDelta := int64(buffer.Len() - oldMemSize) + for _, byItem := range byItems { + fdt, _ := byItem.Expr.Eval(row) + datumMem := aggfuncs.GetDatumMemSize(&fdt) + memDelta += datumMem + } + memDeltas = append(memDeltas, memDelta) + } + return memDeltas, nil +} + +func groupConcatDistinctMultiArgsUpdateMemDeltaGens(srcChk *chunk.Chunk, dataType []*types.FieldType, byItems []*util.ByItems) (memDeltas []int64, err error) { + valSet := set.NewStringSet() + buffer := new(bytes.Buffer) + valsBuf := new(bytes.Buffer) + var encodeBytesBuffer []byte + for i := 0; i < srcChk.NumRows(); i++ { + row := srcChk.GetRow(i) + if row.IsNull(0) { + memDeltas = append(memDeltas, int64(0)) + continue + } + valsBuf.Reset() + encodeBytesBuffer = encodeBytesBuffer[:0] + for j := 0; j < len(dataType); j++ { + curVal := row.GetString(j) + encodeBytesBuffer = codec.EncodeBytes(encodeBytesBuffer, hack.Slice(curVal)) + valsBuf.WriteString(curVal) + } + joinedVal := string(encodeBytesBuffer) + if valSet.Exist(joinedVal) { + memDeltas = append(memDeltas, int64(0)) + continue + } + valSet.Insert(joinedVal) + oldMemSize := buffer.Cap() + if i != 0 { + buffer.WriteString(separator) + } + buffer.WriteString(valsBuf.String()) + memDelta := int64(len(joinedVal) + (buffer.Cap() - oldMemSize)) + memDeltas = append(memDeltas, memDelta) + } + return memDeltas, nil +} + +func groupConcatDistinctOrderMultiArgsUpdateMemDeltaGens(srcChk *chunk.Chunk, dataType []*types.FieldType, byItems []*util.ByItems) (memDeltas []int64, err error) { + valSet := set.NewStringSet() + buffer := new(bytes.Buffer) + valsBuf := new(bytes.Buffer) + var encodeBytesBuffer []byte + for i := 0; i < srcChk.NumRows(); i++ { + row := srcChk.GetRow(i) + if row.IsNull(0) { + memDeltas = append(memDeltas, int64(0)) + continue + } + valsBuf.Reset() + encodeBytesBuffer = encodeBytesBuffer[:0] + for j := 0; j < len(dataType); j++ { + curVal := row.GetString(j) + encodeBytesBuffer = codec.EncodeBytes(encodeBytesBuffer, hack.Slice(curVal)) + valsBuf.WriteString(curVal) + } + joinedVal := string(encodeBytesBuffer) + if valSet.Exist(joinedVal) { + memDeltas = append(memDeltas, int64(0)) + continue + } + oldMemSize := buffer.Len() + if i != 0 { + buffer.WriteString(separator) + } + valSet.Insert(joinedVal) + buffer.WriteString(valsBuf.String()) + memDelta := int64(len(joinedVal) + (buffer.Len() - oldMemSize)) + for _, byItem := range byItems { + fdt, _ := byItem.Expr.Eval(row) + datumMem := aggfuncs.GetDatumMemSize(&fdt) + memDelta += datumMem + } + memDeltas = append(memDeltas, memDelta) + } + return memDeltas, nil +} diff --git a/executor/aggfuncs/func_json_objectagg_test.go b/executor/aggfuncs/func_json_objectagg_test.go index 2a424b6831109..aea5d0a29581e 100644 --- a/executor/aggfuncs/func_json_objectagg_test.go +++ b/executor/aggfuncs/func_json_objectagg_test.go @@ -113,10 +113,7 @@ func (s *testSuite) TestMemJsonObjectagg(c *C) { argCombines = append(argCombines, argTypes) } } - - var tests []multiArgsAggMemTest numRows := 5 - for k := 0; k < len(argCombines); k++ { entries := make(map[string]interface{}) @@ -143,12 +140,12 @@ func (s *testSuite) TestMemJsonObjectagg(c *C) { } } - aggTest := buildMultiArgsAggMemTester(ast.AggFuncJsonObjectAgg, argTypes, mysql.TypeJSON, numRows, aggfuncs.DefPartialResult4JsonObjectAgg, defaultMultiArgsMemDeltaGens, nil, json.CreateBinary(entries)) - - tests = append(tests, aggTest) - } - - for _, test := range tests { - s.testMultiArgsAggMemFunc(c, test) + tests := []multiArgsAggMemTest{ + buildMultiArgsAggMemTester(ast.AggFuncJsonObjectAgg, argTypes, mysql.TypeJSON, numRows, aggfuncs.DefPartialResult4JsonObjectAgg, defaultMultiArgsMemDeltaGens, true), + buildMultiArgsAggMemTester(ast.AggFuncJsonObjectAgg, argTypes, mysql.TypeJSON, numRows, aggfuncs.DefPartialResult4JsonObjectAgg, defaultMultiArgsMemDeltaGens, false), + } + for _, test := range tests { + s.testMultiArgsAggMemFunc(c, test) + } } } diff --git a/executor/analyze.go b/executor/analyze.go old mode 100755 new mode 100644 diff --git a/executor/builder.go b/executor/builder.go index cbedfc175399b..9454f207bb3eb 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2482,7 +2482,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea if containsLimit(dagReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, ts.Desc) } else { - e.feedback = statistics.NewQueryFeedback(getPhysicalTableID(tbl), ts.Hist, int64(ts.StatsCount()), ts.Desc) + e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.ctx, tbl), ts.Hist, int64(ts.StatsCount()), ts.Desc) } collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(ts.Ranges)) if !collect { @@ -2738,7 +2738,11 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plannercore.PhysicalIndexRea if containsLimit(dagReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) } else { - e.feedback = statistics.NewQueryFeedback(e.physicalTableID, is.Hist, int64(is.StatsCount()), is.Desc) + tblID := e.physicalTableID + if b.ctx.GetSessionVars().UseDynamicPartitionPrune() { + tblID = e.table.Meta().ID + } + e.feedback = statistics.NewQueryFeedback(tblID, is.Hist, int64(is.StatsCount()), is.Desc) } collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(is.Ranges)) if !collect { @@ -2875,7 +2879,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) } else { - e.feedback = statistics.NewQueryFeedback(getPhysicalTableID(tbl), is.Hist, int64(is.StatsCount()), is.Desc) + e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.ctx, tbl), is.Hist, int64(is.StatsCount()), is.Desc) } // Do not collect the feedback for table request. collectTable := false @@ -3779,6 +3783,13 @@ func getPhysicalTableID(t table.Table) int64 { return t.Meta().ID } +func getFeedbackStatsTableID(ctx sessionctx.Context, t table.Table) int64 { + if p, ok := t.(table.PhysicalTable); ok && !ctx.GetSessionVars().UseDynamicPartitionPrune() { + return p.GetPhysicalID() + } + return t.Meta().ID +} + func (b *executorBuilder) buildAdminShowTelemetry(v *plannercore.AdminShowTelemetry) Executor { return &AdminShowTelemetryExec{baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID())} } diff --git a/executor/ddl.go b/executor/ddl.go index ae2d685a65ccf..e731d658c5d36 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -122,7 +122,8 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { err = e.executeCreateSequence(x) case *ast.DropSequenceStmt: err = e.executeDropSequence(x) - + case *ast.AlterSequenceStmt: + err = e.executeAlterSequence(x) } if err != nil { // If the owner return ErrTableNotExists error when running this DDL, it may be caused by schema changed, @@ -608,3 +609,7 @@ func (e *DDLExec) executeRepairTable(s *ast.RepairTableStmt) error { func (e *DDLExec) executeCreateSequence(s *ast.CreateSequenceStmt) error { return domain.GetDomain(e.ctx).DDL().CreateSequence(e.ctx, s) } + +func (e *DDLExec) executeAlterSequence(s *ast.AlterSequenceStmt) error { + return domain.GetDomain(e.ctx).DDL().AlterSequence(e.ctx, s) +} diff --git a/executor/ddl_test.go b/executor/ddl_test.go index eec049e51f3d1..eca31b0c5c525 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -1426,3 +1426,16 @@ func (s *testRecoverTable) TestRenameTable(c *C) { tk.MustExec("drop database rename1") tk.MustExec("drop database rename2") } + +func (s *testSuite6) TestAutoIncrementColumnErrorMessage(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Test create an exist database + _, err := tk.Exec("CREATE database test") + c.Assert(err, NotNil) + + tk.MustExec("CREATE TABLE t1 (t1_id INT NOT NULL AUTO_INCREMENT PRIMARY KEY);") + + _, err = tk.Exec("CREATE INDEX idx1 ON t1 ((t1_id + t1_id));") + c.Assert(err.Error(), Equals, ddl.ErrExpressionIndexCanNotRefer.GenWithStackByArgs("idx1").Error()) +} diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index bca52e2632bb5..a95f12b69788c 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -16,6 +16,7 @@ package executor import ( "context" "crypto/tls" + "time" . "github.com/pingcap/check" "github.com/pingcap/failpoint" @@ -390,3 +391,19 @@ func (s *testExecSerialSuite) TestSortSpillDisk(c *C) { err = exec.Close() c.Assert(err, IsNil) } + +func (s *pkgTestSuite) TestSlowQueryRuntimeStats(c *C) { + stats := &slowQueryRuntimeStats{ + totalFileNum: 2, + readFileNum: 2, + readFile: time.Second, + initialize: time.Millisecond, + readFileSize: 1024 * 1024 * 1024, + parseLog: int64(time.Millisecond * 100), + concurrent: 15, + } + c.Assert(stats.String(), Equals, "initialize: 1ms, read_file: 1s, parse_log: {time:100ms, concurrency:15}, total_file: 2, read_file: 2, read_size: 1024 MB") + c.Assert(stats.String(), Equals, stats.Clone().String()) + stats.Merge(stats.Clone()) + c.Assert(stats.String(), Equals, "initialize: 2ms, read_file: 2s, parse_log: {time:200ms, concurrency:15}, total_file: 4, read_file: 4, read_size: 2 GB") +} diff --git a/executor/executor_test.go b/executor/executor_test.go index a33b72ff75655..75b700129db57 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -4053,6 +4053,32 @@ func (s *testSuite3) TestDoSubquery(c *C) { c.Assert(r, IsNil, Commentf("result of Do not empty")) } +func (s *testSuite3) TestSubqueryTableAlias(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + + tk.MustExec("set sql_mode = ''") + tk.MustGetErrCode("select a, b from (select 1 a) ``, (select 2 b) ``;", mysql.ErrDerivedMustHaveAlias) + tk.MustGetErrCode("select a, b from (select 1 a) `x`, (select 2 b) `x`;", mysql.ErrNonuniqTable) + tk.MustGetErrCode("select a, b from (select 1 a), (select 2 b);", mysql.ErrDerivedMustHaveAlias) + // ambiguous column name + tk.MustGetErrCode("select a from (select 1 a) ``, (select 2 a) ``;", mysql.ErrDerivedMustHaveAlias) + tk.MustGetErrCode("select a from (select 1 a) `x`, (select 2 a) `x`;", mysql.ErrNonuniqTable) + tk.MustGetErrCode("select x.a from (select 1 a) `x`, (select 2 a) `x`;", mysql.ErrNonuniqTable) + tk.MustGetErrCode("select a from (select 1 a), (select 2 a);", mysql.ErrDerivedMustHaveAlias) + + tk.MustExec("set sql_mode = 'oracle';") + tk.MustQuery("select a, b from (select 1 a) ``, (select 2 b) ``;").Check(testkit.Rows("1 2")) + tk.MustQuery("select a, b from (select 1 a) `x`, (select 2 b) `x`;").Check(testkit.Rows("1 2")) + tk.MustQuery("select a, b from (select 1 a), (select 2 b);").Check(testkit.Rows("1 2")) + // ambiguous column name + tk.MustGetErrCode("select a from (select 1 a) ``, (select 2 a) ``;", mysql.ErrNonUniq) + tk.MustGetErrCode("select a from (select 1 a) `x`, (select 2 a) `x`;", mysql.ErrNonUniq) + tk.MustGetErrCode("select x.a from (select 1 a) `x`, (select 2 a) `x`;", mysql.ErrNonUniq) + tk.MustGetErrCode("select a from (select 1 a), (select 2 a);", mysql.ErrNonUniq) +} + func (s *testSerialSuite) TestTSOFail(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) @@ -6527,3 +6553,27 @@ func (s *testSerialSuite) TestCoprocessorOOMAction(c *C) { se.Close() } } + +func (s *testSuite) TestIssue20237(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t, s") + tk.MustExec("create table t(a date, b float)") + tk.MustExec("create table s(b float)") + tk.MustExec(`insert into t values(NULL,-37), ("2011-11-04",105), ("2013-03-02",-22), ("2006-07-02",-56), (NULL,124), (NULL,111), ("2018-03-03",-5);`) + tk.MustExec(`insert into s values(-37),(105),(-22),(-56),(124),(105),(111),(-5);`) + tk.MustQuery(`select count(distinct t.a, t.b) from t join s on t.b= s.b;`).Check(testkit.Rows("4")) +} + +func (s *testSerialSuite) TestIssue19148(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a decimal(16, 2));") + tk.MustExec("select * from t where a > any_value(a);") + ctx := tk.Se.(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + tblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + c.Assert(int(tblInfo.Meta().Columns[0].Flag), Equals, 0) +} diff --git a/executor/explain.go b/executor/explain.go index 2e12fd9d417b2..5547573272c10 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -28,6 +28,7 @@ type ExplainExec struct { explain *core.Explain analyzeExec Executor + executed bool rows [][]string cursor int } @@ -71,37 +72,33 @@ func (e *ExplainExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } -func (e *ExplainExec) generateExplainInfo(ctx context.Context) (rows [][]string, err error) { - closed := false - defer func() { - if !closed && e.analyzeExec != nil { - err = e.analyzeExec.Close() - closed = true - } - }() - if e.analyzeExec != nil { +func (e *ExplainExec) executeAnalyzeExec(ctx context.Context) (err error) { + if e.analyzeExec != nil && !e.executed { + defer func() { + err1 := e.analyzeExec.Close() + if err1 != nil { + if err != nil { + err = errors.New(err.Error() + ", " + err1.Error()) + } else { + err = err1 + } + } + }() + e.executed = true chk := newFirstChunk(e.analyzeExec) - var nextErr, closeErr error for { - nextErr = Next(ctx, e.analyzeExec, chk) - if nextErr != nil || chk.NumRows() == 0 { + err = Next(ctx, e.analyzeExec, chk) + if err != nil || chk.NumRows() == 0 { break } } - closeErr = e.analyzeExec.Close() - closed = true - if nextErr != nil { - if closeErr != nil { - err = errors.New(nextErr.Error() + ", " + closeErr.Error()) - } else { - err = nextErr - } - } else if closeErr != nil { - err = closeErr - } - if err != nil { - return nil, err - } + } + return err +} + +func (e *ExplainExec) generateExplainInfo(ctx context.Context) (rows [][]string, err error) { + if err = e.executeAnalyzeExec(ctx); err != nil { + return nil, err } if err = e.explain.RenderResult(); err != nil { return nil, err @@ -111,3 +108,16 @@ func (e *ExplainExec) generateExplainInfo(ctx context.Context) (rows [][]string, } return e.explain.Rows, nil } + +// getAnalyzeExecToExecutedNoDelay gets the analyze DML executor to execute in handleNoDelay function. +// For explain analyze insert/update/delete statement, the analyze executor should be executed in handleNoDelay +// function and then commit transaction if needed. +// Otherwise, in autocommit transaction, the table record change of analyze executor(insert/update/delete...) +// will not be committed. +func (e *ExplainExec) getAnalyzeExecToExecutedNoDelay() Executor { + if e.analyzeExec != nil && !e.executed && e.analyzeExec.Schema().Len() == 0 { + e.executed = true + return e.analyzeExec + } + return nil +} diff --git a/executor/explain_test.go b/executor/explain_test.go index b73d33e4870f8..e8e108e3580c3 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -93,14 +93,15 @@ func (s *testSuite1) TestExplainWrite(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int)") - tk.MustExec("explain analyze insert into t select 1") + tk.MustQuery("explain analyze insert into t select 1") tk.MustQuery("select * from t").Check(testkit.Rows("1")) - tk.MustExec("explain analyze update t set a=2 where a=1") + tk.MustQuery("explain analyze update t set a=2 where a=1") tk.MustQuery("select * from t").Check(testkit.Rows("2")) - tk.MustExec("explain insert into t select 1") + tk.MustQuery("explain insert into t select 1") tk.MustQuery("select * from t").Check(testkit.Rows("2")) - tk.MustExec("explain analyze insert into t select 1") - tk.MustQuery("select * from t order by a").Check(testkit.Rows("1", "2")) + tk.MustQuery("explain analyze insert into t select 1") + tk.MustQuery("explain analyze replace into t values (3)") + tk.MustQuery("select * from t order by a").Check(testkit.Rows("1", "2", "3")) } func (s *testSuite1) TestExplainAnalyzeMemory(c *C) { diff --git a/executor/explain_unit_test.go b/executor/explain_unit_test.go index fa9b13bc66e48..1276575f47c28 100644 --- a/executor/explain_unit_test.go +++ b/executor/explain_unit_test.go @@ -81,6 +81,10 @@ func TestExplainAnalyzeInvokeNextAndClose(t *testing.T) { t.Errorf(err.Error()) } // mockErrorOperator panic + explainExec = &ExplainExec{ + baseExecutor: baseExec, + explain: nil, + } mockOpr = mockErrorOperator{baseExec, true, false} explainExec.analyzeExec = &mockOpr defer func() { diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 73396875e6659..64c6aa36ed91b 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -95,6 +95,7 @@ type innerMergeCtx struct { type lookUpMergeJoinTask struct { outerResult *chunk.List + outerMatch [][]bool outerOrderIdx []chunk.RowPtr innerResult *chunk.Chunk @@ -432,25 +433,23 @@ func (imw *innerMergeWorker) run(ctx context.Context, wg *sync.WaitGroup, cancel func (imw *innerMergeWorker) handleTask(ctx context.Context, task *lookUpMergeJoinTask) (err error) { numOuterChks := task.outerResult.NumChunks() - var outerMatch [][]bool if imw.outerMergeCtx.filter != nil { - outerMatch = make([][]bool, numOuterChks) + task.outerMatch = make([][]bool, numOuterChks) for i := 0; i < numOuterChks; i++ { chk := task.outerResult.GetChunk(i) - outerMatch[i] = make([]bool, chk.NumRows()) - outerMatch[i], err = expression.VectorizedFilter(imw.ctx, imw.outerMergeCtx.filter, chunk.NewIterator4Chunk(chk), outerMatch[i]) + task.outerMatch[i] = make([]bool, chk.NumRows()) + task.outerMatch[i], err = expression.VectorizedFilter(imw.ctx, imw.outerMergeCtx.filter, chunk.NewIterator4Chunk(chk), task.outerMatch[i]) if err != nil { return err } } } + task.memTracker.Consume(int64(cap(task.outerMatch))) task.outerOrderIdx = make([]chunk.RowPtr, 0, task.outerResult.Len()) for i := 0; i < numOuterChks; i++ { numRow := task.outerResult.GetChunk(i).NumRows() for j := 0; j < numRow; j++ { - if len(outerMatch) == 0 || outerMatch[i][j] { - task.outerOrderIdx = append(task.outerOrderIdx, chunk.RowPtr{ChkIdx: uint32(i), RowIdx: uint32(j)}) - } + task.outerOrderIdx = append(task.outerOrderIdx, chunk.RowPtr{ChkIdx: uint32(i), RowIdx: uint32(j)}) } } task.memTracker.Consume(int64(cap(task.outerOrderIdx))) @@ -653,8 +652,11 @@ func (imw *innerMergeWorker) constructDatumLookupKeys(task *lookUpMergeJoinTask) return dLookUpKeys, nil } -func (imw *innerMergeWorker) constructDatumLookupKey(task *lookUpMergeJoinTask, rowIdx chunk.RowPtr) (*indexJoinLookUpContent, error) { - outerRow := task.outerResult.GetRow(rowIdx) +func (imw *innerMergeWorker) constructDatumLookupKey(task *lookUpMergeJoinTask, idx chunk.RowPtr) (*indexJoinLookUpContent, error) { + if task.outerMatch != nil && !task.outerMatch[idx.ChkIdx][idx.RowIdx] { + return nil, nil + } + outerRow := task.outerResult.GetRow(idx) sc := imw.ctx.GetSessionVars().StmtCtx keyLen := len(imw.keyCols) dLookupKey := make([]types.Datum, 0, keyLen) @@ -688,7 +690,7 @@ func (imw *innerMergeWorker) constructDatumLookupKey(task *lookUpMergeJoinTask, } dLookupKey = append(dLookupKey, innerValue) } - return &indexJoinLookUpContent{keys: dLookupKey, row: task.outerResult.GetRow(rowIdx)}, nil + return &indexJoinLookUpContent{keys: dLookupKey, row: task.outerResult.GetRow(idx)}, nil } func (imw *innerMergeWorker) dedupDatumLookUpKeys(lookUpContents []*indexJoinLookUpContent) []*indexJoinLookUpContent { diff --git a/executor/index_lookup_merge_join_test.go b/executor/index_lookup_merge_join_test.go index f52b652d346a9..c0bb253cec963 100644 --- a/executor/index_lookup_merge_join_test.go +++ b/executor/index_lookup_merge_join_test.go @@ -134,3 +134,15 @@ func (s *testSuiteWithData) TestIndexJoinOnSinglePartitionTable(c *C) { c.Assert(strings.Index(rows[0], "IndexJoin"), Equals, 0) } } + +func (s *testSuite9) TestIssue20400(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t, s") + tk.MustExec("create table s(a int, index(a))") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1)") + tk.MustQuery("select /*+ hash_join(t,s)*/ * from t left join s on t.a=s.a and t.a>1").Check( + testkit.Rows("1 ")) + tk.MustQuery("select /*+ inl_merge_join(t,s)*/ * from t left join s on t.a=s.a and t.a>1").Check( + testkit.Rows("1 ")) +} diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 137a29b8cde60..59e5f4b3b2d5e 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -451,7 +451,7 @@ func (e *memtableRetriever) setDataFromTables(ctx sessionctx.Context, schemas [] } var rowCount, dataLength, indexLength uint64 - if table.GetPartitionInfo() == nil { + if table.GetPartitionInfo() == nil || ctx.GetSessionVars().UseDynamicPartitionPrune() { rowCount = tableRowsMap[table.ID] dataLength, indexLength = getDataAndIndexLength(table, table.ID, rowCount, colLengthMap) } else { diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index e5328b2be1f06..670afa8e8b49d 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -418,37 +418,39 @@ func (s *testInfoschemaTableSerialSuite) TestPartitionsTable(c *C) { is := do.InfoSchema() tk := testkit.NewTestKit(c, s.store) - tk.MustExec("USE test;") - tk.MustExec("DROP TABLE IF EXISTS `test_partitions`;") - tk.MustExec(`CREATE TABLE test_partitions (a int, b int, c varchar(5), primary key(a), index idx(c)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16));`) - err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - tk.MustExec(`insert into test_partitions(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e");`) - - tk.MustQuery("select PARTITION_NAME, PARTITION_DESCRIPTION from information_schema.PARTITIONS where table_name='test_partitions';").Check( - testkit.Rows("" + - "p0 6]\n" + - "[p1 11]\n" + - "[p2 16")) - - tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.PARTITIONS where table_name='test_partitions';").Check( - testkit.Rows("" + - "0 0 0 0]\n" + - "[0 0 0 0]\n" + - "[0 0 0 0")) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) - tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.PARTITIONS where table_name='test_partitions';").Check( - testkit.Rows("" + - "1 18 18 2]\n" + - "[1 18 18 2]\n" + - "[1 18 18 2")) + testkit.WithPruneMode(tk, variable.StaticOnly, func() { + c.Assert(h.RefreshVars(), IsNil) + tk.MustExec("DROP TABLE IF EXISTS `test_partitions`;") + tk.MustExec(`CREATE TABLE test_partitions (a int, b int, c varchar(5), primary key(a), index idx(c)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16));`) + err := h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) + tk.MustExec(`insert into test_partitions(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e");`) + + tk.MustQuery("select PARTITION_NAME, PARTITION_DESCRIPTION from information_schema.PARTITIONS where table_name='test_partitions';").Check( + testkit.Rows("" + + "p0 6]\n" + + "[p1 11]\n" + + "[p2 16")) + + tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.PARTITIONS where table_name='test_partitions';").Check( + testkit.Rows("" + + "0 0 0 0]\n" + + "[0 0 0 0]\n" + + "[0 0 0 0")) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + c.Assert(h.Update(is), IsNil) + tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.PARTITIONS where table_name='test_partitions';").Check( + testkit.Rows("" + + "1 18 18 2]\n" + + "[1 18 18 2]\n" + + "[1 18 18 2")) + }) // Test for table has no partitions. tk.MustExec("DROP TABLE IF EXISTS `test_partitions_1`;") tk.MustExec(`CREATE TABLE test_partitions_1 (a int, b int, c varchar(5), primary key(a), index idx(c));`) - err = h.HandleDDLEvent(<-h.DDLEventCh()) + err := h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(err, IsNil) tk.MustExec(`insert into test_partitions_1(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e");`) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) diff --git a/executor/inspection_profile.go b/executor/inspection_profile.go index 56bf9153fa10f..397c276634f88 100644 --- a/executor/inspection_profile.go +++ b/executor/inspection_profile.go @@ -323,11 +323,16 @@ func NewProfileBuilder(sctx sessionctx.Context, start, end time.Time, tp string) // Collect uses to collect the related metric information. func (pb *profileBuilder) Collect() error { - pb.buf.WriteString(fmt.Sprintf(`digraph "%s" {`, "tidb_profile")) - pb.buf.WriteByte('\n') - pb.buf.WriteString(`node [style=filled fillcolor="#f8f8f8"]`) - pb.buf.WriteByte('\n') - err := pb.addMetricTree(pb.genTiDBQueryTree(), "tidb_query") + tidbQuery := pb.genTiDBQueryTree() + err := pb.init(tidbQuery, "tidb_query") + if err != nil { + return err + } + err = pb.traversal(tidbQuery) + if err != nil { + return err + } + err = pb.traversal(pb.genTiDBGCTree()) if err != nil { return err } @@ -350,8 +355,8 @@ func (pb *profileBuilder) getNameID(name string) uint64 { return id } -func (pb *profileBuilder) addMetricTree(root *metricNode, name string) error { - if root == nil { +func (pb *profileBuilder) init(total *metricNode, name string) error { + if total == nil { return nil } tp := "total_time" @@ -361,9 +366,13 @@ func (pb *profileBuilder) addMetricTree(root *metricNode, name string) error { case metricValueCnt: tp = "total_count" } + pb.buf.WriteString(fmt.Sprintf(`digraph "%s" {`, "tidb_profile")) + pb.buf.WriteByte('\n') + pb.buf.WriteString(`node [style=filled fillcolor="#f8f8f8"]`) + pb.buf.WriteByte('\n') pb.buf.WriteString(fmt.Sprintf(`subgraph %[1]s { "%[1]s" [shape=box fontsize=16 label="Type: %[1]s\lTime: %s\lDuration: %s\l"] }`, name+"_"+tp, pb.start.String(), pb.end.Sub(pb.start).String())) pb.buf.WriteByte('\n') - v, err := pb.GetTotalValue(root) + v, err := pb.GetTotalValue(total) if err != nil { return err } @@ -372,7 +381,7 @@ func (pb *profileBuilder) addMetricTree(root *metricNode, name string) error { } else { pb.totalValue = 1 } - return pb.traversal(root) + return nil } func (pb *profileBuilder) GetTotalValue(root *metricNode) (float64, error) { @@ -622,6 +631,21 @@ func (pb *profileBuilder) dotColor(score float64, isBackground bool) string { return fmt.Sprintf("#%02x%02x%02x", uint8(r*255.0), uint8(g*255.0), uint8(b*255.0)) } +func (pb *profileBuilder) genTiDBGCTree() *metricNode { + tidbGC := &metricNode{ + table: "tidb_gc", + isPartOfParent: true, + label: []string{"stage"}, + children: []*metricNode{ + { + table: "tidb_kv_request", + isPartOfParent: true, + }, + }, + } + return tidbGC +} + func (pb *profileBuilder) genTiDBQueryTree() *metricNode { tidbKVRequest := &metricNode{ table: "tidb_kv_request", @@ -689,6 +713,10 @@ func (pb *profileBuilder) genTiDBQueryTree() *metricNode { }, }, }, + { + table: "tikv_gc_tasks", + label: []string{"task"}, + }, }, }, }, @@ -713,6 +741,9 @@ func (pb *profileBuilder) genTiDBQueryTree() *metricNode { { table: "tidb_owner_handle_syncer", }, + { + table: "tidb_meta_operation", + }, }, }, }, diff --git a/executor/inspection_result.go b/executor/inspection_result.go index b14284b3889c8..4d8bd69ca3edb 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -142,7 +142,7 @@ func (e *inspectionResultRetriever) retrieve(ctx context.Context, sctx sessionct e.instanceToStatusAddress = make(map[string]string) e.statusToInstanceAddress = make(map[string]string) sql := "select instance,status_address from information_schema.cluster_info;" - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithContext(ctx, sql) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("get cluster info failed: %v", err)) } @@ -213,7 +213,7 @@ func (c configInspection) inspect(ctx context.Context, sctx sessionctx.Context, return results } -func (configInspection) inspectDiffConfig(_ context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { +func (configInspection) inspectDiffConfig(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { // check the configuration consistent ignoreConfigKey := []string{ // TiDB @@ -249,14 +249,14 @@ func (configInspection) inspectDiffConfig(_ context.Context, sctx sessionctx.Con } sql := fmt.Sprintf("select type, `key`, count(distinct value) as c from information_schema.cluster_config where `key` not in ('%s') group by type, `key` having c > 1", strings.Join(ignoreConfigKey, "','")) - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithContext(ctx, sql) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check configuration consistency failed: %v", err)) } generateDetail := func(tp, item string) string { query := fmt.Sprintf("select value, instance from information_schema.cluster_config where type='%s' and `key`='%s';", tp, item) - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(query) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithContext(ctx, query) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check configuration consistency failed: %v", err)) return fmt.Sprintf("the cluster has different config value of %[2]s, execute the sql to see more detail: select * from information_schema.cluster_config where type='%[1]s' and `key`='%[2]s'", @@ -324,7 +324,7 @@ func (c configInspection) inspectCheckConfig(ctx context.Context, sctx sessionct } sql := fmt.Sprintf("select instance from information_schema.cluster_config where type = '%s' and `key` = '%s' and value = '%s'", cas.tp, cas.key, cas.value) - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithContext(ctx, sql) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check configuration in reason failed: %v", err)) } @@ -437,10 +437,10 @@ func (configInspection) convertReadableSizeToByteSize(sizeStr string) (uint64, e return uint64(size) * rate, nil } -func (versionInspection) inspect(_ context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { +func (versionInspection) inspect(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { // check the configuration consistent sql := "select type, count(distinct git_hash) as c from information_schema.cluster_info group by type having c > 1;" - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithContext(ctx, sql) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check version consistency failed: %v", err)) } diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 1f5a88e516982..bf17e3520dbb6 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/set" "go.uber.org/zap" @@ -53,9 +54,12 @@ type dummyCloser struct{} func (dummyCloser) close() error { return nil } +func (dummyCloser) getRuntimeStats() execdetails.RuntimeStats { return nil } + type memTableRetriever interface { retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) close() error + getRuntimeStats() execdetails.RuntimeStats } // MemTableReaderExec executes memTable information retrieving from the MemTable components @@ -127,6 +131,9 @@ func (e *MemTableReaderExec) Next(ctx context.Context, req *chunk.Chunk) error { // Close implements the Executor Close interface. func (e *MemTableReaderExec) Close() error { + if stats := e.retriever.getRuntimeStats(); stats != nil && e.runtimeStats != nil { + e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, stats) + } return e.retriever.close() } @@ -661,3 +668,7 @@ func (e *clusterLogRetriever) close() error { } return nil } + +func (e *clusterLogRetriever) getRuntimeStats() execdetails.RuntimeStats { + return nil +} diff --git a/executor/show.go b/executor/show.go index 521532988b71e..d06a135418970 100644 --- a/executor/show.go +++ b/executor/show.go @@ -474,6 +474,9 @@ func (e *ShowExec) fetchShowColumns(ctx context.Context) error { if idx >= 0 { col.FieldType = *viewSchema.Columns[idx].GetType() } + if col.Tp == mysql.TypeVarString { + col.Tp = mysql.TypeVarchar + } } } for _, col := range cols { diff --git a/executor/show_stats.go b/executor/show_stats.go index 79b1c97a7dfb7..ac6a1542e98ab 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -31,7 +31,7 @@ func (e *ShowExec) fetchShowStatsMeta() error { for _, db := range dbs { for _, tbl := range db.Tables { pi := tbl.GetPartitionInfo() - if pi == nil { + if pi == nil || e.ctx.GetSessionVars().UseDynamicPartitionPrune() { e.appendTableForStatsMeta(db.Name.O, tbl.Name.O, "", h.GetTableStats(tbl)) } else { for _, def := range pi.Definitions { @@ -64,7 +64,7 @@ func (e *ShowExec) fetchShowStatsHistogram() error { for _, db := range dbs { for _, tbl := range db.Tables { pi := tbl.GetPartitionInfo() - if pi == nil { + if pi == nil || e.ctx.GetSessionVars().UseDynamicPartitionPrune() { e.appendTableForStatsHistograms(db.Name.O, tbl.Name.O, "", h.GetTableStats(tbl)) } else { for _, def := range pi.Definitions { @@ -119,7 +119,7 @@ func (e *ShowExec) fetchShowStatsBuckets() error { for _, db := range dbs { for _, tbl := range db.Tables { pi := tbl.GetPartitionInfo() - if pi == nil { + if pi == nil || e.ctx.GetSessionVars().UseDynamicPartitionPrune() { if err := e.appendTableForStatsBuckets(db.Name.O, tbl.Name.O, "", h.GetTableStats(tbl)); err != nil { return err } @@ -199,7 +199,7 @@ func (e *ShowExec) fetchShowStatsHealthy() { for _, db := range dbs { for _, tbl := range db.Tables { pi := tbl.GetPartitionInfo() - if pi == nil { + if pi == nil || e.ctx.GetSessionVars().UseDynamicPartitionPrune() { e.appendTableForStatsHealthy(db.Name.O, tbl.Name.O, "", h.GetTableStats(tbl)) } else { for _, def := range pi.Definitions { diff --git a/executor/slow_query.go b/executor/slow_query.go old mode 100644 new mode 100755 index fab505f431910..ca573c6370510 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -18,12 +18,14 @@ import ( "context" "fmt" "io" + "io/ioutil" "os" "path/filepath" "sort" "strconv" "strings" "sync" + "sync/atomic" "time" "github.com/pingcap/errors" @@ -41,6 +43,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/plancodec" "go.uber.org/zap" ) @@ -57,6 +60,7 @@ type slowQueryRetriever struct { checker *slowLogChecker parsedSlowLogCh chan parsedSlowLog + stats *slowQueryRuntimeStats } func (e *slowQueryRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { @@ -98,6 +102,7 @@ func (e *slowQueryRetriever) initialize(sctx sessionctx.Context) error { hasProcessPriv: hasProcessPriv, user: sctx.GetSessionVars().User, } + e.stats = &slowQueryRuntimeStats{} if e.extractor != nil { e.checker.enableTimeCheck = e.extractor.Enable e.checker.startTime = types.NewTime(types.FromGoTime(e.extractor.StartTime), mysql.TypeDatetime, types.MaxFsp) @@ -123,14 +128,31 @@ type parsedSlowLog struct { err error } +func (e *slowQueryRetriever) getNextFile() *os.File { + if e.fileIdx >= len(e.files) { + return nil + } + file := e.files[e.fileIdx].file + e.fileIdx++ + if e.stats != nil { + stat, err := file.Stat() + if err == nil { + // ignore the err will be ok. + e.stats.readFileSize += stat.Size() + e.stats.readFileNum++ + } + } + return file +} + func (e *slowQueryRetriever) parseDataForSlowLog(ctx context.Context, sctx sessionctx.Context) { - if len(e.files) == 0 { - close(e.parsedSlowLogCh) + defer close(e.parsedSlowLogCh) + file := e.getNextFile() + if file == nil { return } - reader := bufio.NewReader(e.files[0].file) + reader := bufio.NewReader(file) e.parseSlowLog(ctx, sctx, reader, 64) - close(e.parsedSlowLogCh) } func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context) ([][]types.Datum, bool, error) { @@ -228,13 +250,13 @@ func (e *slowQueryRetriever) getBatchLog(reader *bufio.Reader, offset *offset, n lineByte, err := getOneLine(reader) if err != nil { if err == io.EOF { - e.fileIdx++ e.fileLine = 0 - if e.fileIdx >= len(e.files) { + file := e.getNextFile() + if file == nil { return log, nil } offset.length = len(log) - reader.Reset(e.files[e.fileIdx].file) + reader.Reset(file) continue } return log, err @@ -256,14 +278,22 @@ func (e *slowQueryRetriever) parseSlowLog(ctx context.Context, sctx sessionctx.C var wg sync.WaitGroup offset := offset{offset: 0, length: 0} // To limit the num of go routine - ch := make(chan int, sctx.GetSessionVars().Concurrency.DistSQLScanConcurrency()) + concurrent := sctx.GetSessionVars().Concurrency.DistSQLScanConcurrency() + ch := make(chan int, concurrent) + if e.stats != nil { + e.stats.concurrent = concurrent + } defer close(ch) for { + startTime := time.Now() log, err := e.getBatchLog(reader, &offset, logNum) - if err != nil { + if err != nil || len(log) == 0 { e.parsedSlowLogCh <- parsedSlowLog{nil, err} break } + if e.stats != nil { + e.stats.readFile += time.Since(startTime) + } start := offset wg.Add(1) ch <- 1 @@ -303,10 +333,14 @@ func getLineIndex(offset offset, index int) int { } func (e *slowQueryRetriever) parseLog(ctx sessionctx.Context, log []string, offset offset) (data [][]types.Datum, err error) { + start := time.Now() defer func() { if r := recover(); r != nil { err = fmt.Errorf("%s", r) } + if e.stats != nil { + atomic.AddInt64(&e.stats.parseLog, int64(time.Since(start))) + } }() failpoint.Inject("errorMockParseSlowLogPanic", func(val failpoint.Value) { if val.(bool) { @@ -704,7 +738,16 @@ type logFile struct { // getAllFiles is used to get all slow-log needed to parse, it is exported for test. func (e *slowQueryRetriever) getAllFiles(sctx sessionctx.Context, logFilePath string) ([]logFile, error) { + totalFileNum := 0 + if e.stats != nil { + startTime := time.Now() + defer func() { + e.stats.initialize = time.Since(startTime) + e.stats.totalFileNum = totalFileNum + }() + } if e.extractor == nil || !e.extractor.Enable { + totalFileNum = 1 file, err := os.Open(logFilePath) if err != nil { if os.IsNotExist(err) { @@ -725,10 +768,11 @@ func (e *slowQueryRetriever) getAllFiles(sctx sessionctx.Context, logFilePath st } return nil } - err := filepath.Walk(logDir, func(path string, info os.FileInfo, err error) error { - if err != nil { - return handleErr(err) - } + files, err := ioutil.ReadDir(logDir) + if err != nil { + return nil, err + } + walkFn := func(path string, info os.FileInfo) error { if info.IsDir() { return nil } @@ -736,6 +780,7 @@ func (e *slowQueryRetriever) getAllFiles(sctx sessionctx.Context, logFilePath st if !strings.HasPrefix(path, prefix) { return nil } + totalFileNum++ file, err := os.OpenFile(path, os.O_RDONLY, os.ModePerm) if err != nil { return handleErr(err) @@ -776,7 +821,13 @@ func (e *slowQueryRetriever) getAllFiles(sctx sessionctx.Context, logFilePath st }) skip = true return nil - }) + } + for _, file := range files { + err := walkFn(filepath.Join(logDir, file.Name()), file) + if err != nil { + return nil, err + } + } // Sort by start time sort.Slice(logFiles, func(i, j int) bool { return logFiles[i].start.Before(logFiles[j].start) @@ -808,55 +859,133 @@ func (e *slowQueryRetriever) getFileStartTime(file *os.File) (time.Time, error) } return t, errors.Errorf("malform slow query file %v", file.Name()) } + +func (e *slowQueryRetriever) getRuntimeStats() execdetails.RuntimeStats { + return e.stats +} + +type slowQueryRuntimeStats struct { + totalFileNum int + readFileNum int + readFile time.Duration + initialize time.Duration + readFileSize int64 + parseLog int64 + concurrent int +} + +// String implements the RuntimeStats interface. +func (s *slowQueryRuntimeStats) String() string { + return fmt.Sprintf("initialize: %s, read_file: %s, parse_log: {time:%s, concurrency:%v}, total_file: %v, read_file: %v, read_size: %s", + s.initialize, s.readFile, time.Duration(s.parseLog), s.concurrent, + s.totalFileNum, s.readFileNum, memory.BytesToString(s.readFileSize)) +} + +// Merge implements the RuntimeStats interface. +func (s *slowQueryRuntimeStats) Merge(rs execdetails.RuntimeStats) { + tmp, ok := rs.(*slowQueryRuntimeStats) + if !ok { + return + } + s.totalFileNum += tmp.totalFileNum + s.readFileNum += tmp.readFileNum + s.readFile += tmp.readFile + s.initialize += tmp.initialize + s.readFileSize += tmp.readFileSize + s.parseLog += tmp.parseLog +} + +// Clone implements the RuntimeStats interface. +func (s *slowQueryRuntimeStats) Clone() execdetails.RuntimeStats { + newRs := *s + return &newRs +} + +// Tp implements the RuntimeStats interface. +func (s *slowQueryRuntimeStats) Tp() int { + return execdetails.TpSlowQueryRuntimeStat +} + func (e *slowQueryRetriever) getFileEndTime(file *os.File) (time.Time, error) { var t time.Time + var tried int stat, err := file.Stat() if err != nil { return t, err } - fileSize := stat.Size() - cursor := int64(0) - line := make([]byte, 0, 64) + endCursor := stat.Size() maxLineNum := 128 - tryGetTime := func(line []byte) string { - for i, j := 0, len(line)-1; i < j; i, j = i+1, j-1 { - line[i], line[j] = line[j], line[i] + for { + lines, readBytes, err := readLastLines(file, endCursor) + if err != nil { + return t, err + } + // read out the file + if readBytes == 0 { + break + } + endCursor -= int64(readBytes) + for i := len(lines) - 1; i >= 0; i-- { + if strings.HasPrefix(lines[i], variable.SlowLogStartPrefixStr) { + return ParseTime(lines[i][len(variable.SlowLogStartPrefixStr):]) + } } - lineStr := string(line) - lineStr = strings.TrimSpace(lineStr) - if strings.HasPrefix(lineStr, variable.SlowLogStartPrefixStr) { - return lineStr[len(variable.SlowLogStartPrefixStr):] + tried += len(lines) + if tried >= maxLineNum { + break } - return "" } + return t, errors.Errorf("invalid slow query file %v", file.Name()) +} + +// Read lines from the end of a file +// endCursor initial value should be the filesize +func readLastLines(file *os.File, endCursor int64) ([]string, int, error) { + var lines []byte + var firstNonNewlinePos int + var cursor = endCursor for { - cursor -= 1 - _, err := file.Seek(cursor, io.SeekEnd) - if err != nil { - return t, err + // stop if we are at the beginning + // check it in the start to avoid read beyond the size + if cursor <= 0 { + break + } + + var size int64 = 4096 + if cursor < size { + size = cursor } + cursor -= size - char := make([]byte, 1) - _, err = file.Read(char) + _, err := file.Seek(cursor, io.SeekStart) if err != nil { - return t, err + return nil, 0, err } - // If find a line. - if cursor != -1 && (char[0] == '\n' || char[0] == '\r') { - if timeStr := tryGetTime(line); len(timeStr) > 0 { - return ParseTime(timeStr) - } - line = line[:0] - maxLineNum -= 1 + chars := make([]byte, size) + _, err = file.Read(chars) + if err != nil { + return nil, 0, err } - line = append(line, char[0]) - if cursor == -fileSize || maxLineNum <= 0 { - if timeStr := tryGetTime(line); len(timeStr) > 0 { - return ParseTime(timeStr) + lines = append(chars, lines...) + + // find first '\n' or '\r' + for i := 0; i < len(chars); i++ { + // reach the line end + // the first newline may be in the line end at the first round + if i >= len(lines)-1 { + break } - return t, errors.Errorf("malform slow query file %v", file.Name()) + if (chars[i] == 10 || chars[i] == 13) && chars[i+1] != 10 && chars[i+1] != 13 { + firstNonNewlinePos = i + 1 + break + } + } + if firstNonNewlinePos > 0 { + break } } + finalStr := string(lines[firstNonNewlinePos:]) + return strings.Split(strings.ReplaceAll(finalStr, "\r\n", "\n"), "\n"), len(finalStr), nil } func (e *slowQueryRetriever) initializeAsyncParsing(ctx context.Context, sctx sessionctx.Context) { diff --git a/executor/split.go b/executor/split.go old mode 100755 new mode 100644 diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index df857488d0cf4..8f590a6fc5ad7 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -1305,7 +1305,7 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express // -inf: 10000000 & 1 == 0 // For uint: // inf: 11111111 & 1 == 1 - // -inf: 00000000 & 0 == 0 + // -inf: 00000000 & 1 == 0 if arg1.Value.GetInt64()&1 == 1 { isPositiveInfinite = true } else { @@ -1344,7 +1344,55 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express return []Expression{NewOne(), NewZero()} } - return []Expression{finalArg0, finalArg1} + return c.refineArgsByUnsignedFlag(ctx, []Expression{finalArg0, finalArg1}) +} + +func (c *compareFunctionClass) refineArgsByUnsignedFlag(ctx sessionctx.Context, args []Expression) []Expression { + // Only handle int cases, cause MySQL declares that `UNSIGNED` is deprecated for FLOAT, DOUBLE and DECIMAL types, + // and support for it would be removed in a future version. + if args[0].GetType().EvalType() != types.ETInt || args[1].GetType().EvalType() != types.ETInt { + return args + } + colArgs := make([]*Column, 2) + constArgs := make([]*Constant, 2) + for i, arg := range args { + switch x := arg.(type) { + case *Constant: + constArgs[i] = x + case *Column: + colArgs[i] = x + case *CorrelatedColumn: + colArgs[i] = &x.Column + } + } + for i := 0; i < 2; i++ { + if con, col := constArgs[1-i], colArgs[i]; con != nil && col != nil { + v, isNull, err := con.EvalInt(ctx, chunk.Row{}) + if err != nil || isNull || v > 0 { + return args + } + if mysql.HasUnsignedFlag(col.RetType.Flag) && mysql.HasNotNullFlag(col.RetType.Flag) && !mysql.HasUnsignedFlag(con.RetType.Flag) { + op := c.op + if i == 1 { + op = symmetricOp[c.op] + } + if v == 0 && (op == opcode.LE || op == opcode.GT || op == opcode.NullEQ || op == opcode.EQ || op == opcode.NE) { + return args + } + // `unsigned_col < 0` equals to `1 < 0`, + // `unsigned_col > -1` equals to `1 > 0`, + // `unsigned_col <= -1` equals to `1 <= 0`, + // `unsigned_col >= 0` equals to `1 >= 0`, + // `unsigned_col == -1` equals to `1 == 0`, + // `unsigned_col != -1` equals to `1 != 0`, + // `unsigned_col <=> -1` equals to `1 <=> 0`, + // so we can replace the column argument with `1`, and the other constant argument with `0`. + args[i], args[1-i] = NewOne(), NewZero() + return args + } + } + } + return args } // getFunction sets compare built-in function signatures for various types. diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index 9e40a70f11e3e..cd9a7a2ca3bf0 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -222,8 +222,9 @@ func (c *anyValueFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err != nil { return nil, err } - args[0].GetType().Flag |= bf.tp.Flag - *bf.tp = *args[0].GetType() + ft := args[0].GetType().Clone() + ft.Flag |= bf.tp.Flag + *bf.tp = *ft var sig builtinFunc switch argTp { case types.ETDecimal: diff --git a/expression/collation.go b/expression/collation.go index e99ace858b89f..d8bae9f8c45c3 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -153,7 +153,7 @@ func deriveCoercibilityForScarlarFunc(sf *ScalarFunction) Coercibility { if _, ok := sysConstFuncs[sf.FuncName.L]; ok { return CoercibilitySysconst } - if !types.IsString(sf.RetType.Tp) { + if sf.RetType.EvalType() != types.ETString { return CoercibilityNumeric } @@ -170,14 +170,14 @@ func deriveCoercibilityForScarlarFunc(sf *ScalarFunction) Coercibility { func deriveCoercibilityForConstant(c *Constant) Coercibility { if c.Value.IsNull() { return CoercibilityIgnorable - } else if !types.IsString(c.RetType.Tp) { + } else if c.RetType.EvalType() != types.ETString { return CoercibilityNumeric } return CoercibilityCoercible } func deriveCoercibilityForColumn(c *Column) Coercibility { - if !types.IsString(c.RetType.Tp) { + if c.RetType.EvalType() != types.ETString { return CoercibilityNumeric } return CoercibilityImplicit diff --git a/expression/constant_fold.go b/expression/constant_fold.go index 8045838d594ef..1b4dcaf00c3ad 100644 --- a/expression/constant_fold.go +++ b/expression/constant_fold.go @@ -15,6 +15,7 @@ package expression import ( "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" @@ -28,6 +29,7 @@ func init() { ast.If: ifFoldHandler, ast.Ifnull: ifNullFoldHandler, ast.Case: caseWhenHandler, + ast.IsNull: isNullHandler, } } @@ -39,6 +41,29 @@ func FoldConstant(expr Expression) Expression { return e } +func isNullHandler(expr *ScalarFunction) (Expression, bool) { + arg0 := expr.GetArgs()[0] + if constArg, isConst := arg0.(*Constant); isConst { + isDeferredConst := constArg.DeferredExpr != nil || constArg.ParamMarker != nil + value, err := expr.Eval(chunk.Row{}) + if err != nil { + // Failed to fold this expr to a constant, print the DEBUG log and + // return the original expression to let the error to be evaluated + // again, in that time, the error is returned to the client. + logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", expr.ExplainInfo()), zap.Error(err)) + return expr, isDeferredConst + } + if isDeferredConst { + return &Constant{Value: value, RetType: expr.RetType, DeferredExpr: expr}, true + } + return &Constant{Value: value, RetType: expr.RetType}, false + } + if mysql.HasNotNullFlag(arg0.GetType().Flag) { + return NewZero(), false + } + return expr, false +} + func ifFoldHandler(expr *ScalarFunction) (Expression, bool) { args := expr.GetArgs() foldedArg0, _ := foldConstant(args[0]) diff --git a/expression/flag_simplify_test.go b/expression/flag_simplify_test.go new file mode 100644 index 0000000000000..cb0ef080e00d6 --- /dev/null +++ b/expression/flag_simplify_test.go @@ -0,0 +1,81 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package expression_test + +import ( + "fmt" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testutil" +) + +var _ = Suite(&testFlagSimplifySuite{}) + +type testFlagSimplifySuite struct { + store kv.Storage + dom *domain.Domain + ctx sessionctx.Context + testData testutil.TestData +} + +func (s *testFlagSimplifySuite) cleanEnv(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show tables") + for _, tb := range r.Rows() { + tableName := tb[0] + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } +} + +func (s *testFlagSimplifySuite) SetUpSuite(c *C) { + var err error + s.store, s.dom, err = newStoreWithBootstrap() + c.Assert(err, IsNil) + s.ctx = mock.NewContext() + s.testData, err = testutil.LoadTestSuiteData("testdata", "flag_simplify") + c.Assert(err, IsNil) +} + +func (s *testFlagSimplifySuite) TearDownSuite(c *C) { + c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) + s.dom.Close() + s.store.Close() +} + +func (s *testFlagSimplifySuite) TestSimplifyExpressionByFlag(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key, a bigint unsigned not null, b bigint unsigned)") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/expression/integration_test.go b/expression/integration_test.go old mode 100755 new mode 100644 index bb97eb95a1f46..5ccf9fc3fd633 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -297,6 +297,15 @@ func (s *testIntegrationSuite) TestConvertToBit(c *C) { tk.MustExec(`insert t1 value ('09-01-01')`) tk.MustExec(`insert t select a from t1`) tk.MustQuery("select a+0 from t").Check(testkit.Rows("20090101000000")) + + // For issue 20118 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a tinyint, b bit(63));") + tk.MustExec("insert ignore into t values(599999999, -1);") + tk.MustQuery("show warnings;").Check(testkit.Rows( + "Warning 1690 constant 599999999 overflows tinyint", + "Warning 1406 Data Too Long, field len 63")) + tk.MustQuery("select * from t;").Check(testkit.Rows("127 \u007f\xff\xff\xff\xff\xff\xff\xff")) } func (s *testIntegrationSuite2) TestMathBuiltin(c *C) { @@ -6704,6 +6713,48 @@ func (s *testIntegrationSerialSuite) TestNewCollationWithClusterIndex(c *C) { tk.MustQuery("select d from t use index(idx) where name=\"aa\"").Check(testkit.Rows("2.11")) } +func (s *testIntegrationSerialSuite) TestNewCollationBinaryFlag(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(2) binary, index (a));") + tk.MustExec("insert into t values ('a ');") + tk.MustQuery("select hex(a) from t;").Check(testkit.Rows("6120")) + tk.MustQuery("select hex(a) from t use index (a);").Check(testkit.Rows("6120")) + + showCreateTable := func(createSQL string) string { + tk.MustExec("drop table if exists t;") + tk.MustExec(createSQL) + s := tk.MustQuery("show create table t;").Rows()[0][1].(string) + return s + } + var sct string + // define case = tuple(table_charset, table_collation, column_charset, column_collation) + // case: (nil, nil, nil, nil) + sct = showCreateTable("create table t(a varchar(10) binary);") + c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"), IsTrue, Commentf(sct)) + // case: (nil, utf8_general_ci, nil, nil) + sct = showCreateTable("create table t(a varchar(10) binary) collate utf8_general_ci;") + c.Assert(strings.Contains(sct, "varchar(10) COLLATE utf8_bin"), IsTrue, Commentf(sct)) + c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci"), IsTrue, Commentf(sct)) + // case: (nil, nil, nil, utf8_general_ci) + sct = showCreateTable("create table t(a varchar(10) binary collate utf8_general_ci);") + c.Assert(strings.Contains(sct, "varchar(10) CHARACTER SET utf8 COLLATE utf8_bin"), IsTrue, Commentf(sct)) + // case: (nil, nil, utf8, utf8_general_ci) + sct = showCreateTable("create table t(a varchar(10) binary charset utf8 collate utf8_general_ci);") + c.Assert(strings.Contains(sct, "varchar(10) CHARACTER SET utf8 COLLATE utf8_general_ci"), IsTrue, Commentf(sct)) + // case: (utf8, utf8_general_ci, utf8mb4, utf8mb4_unicode_ci) + sct = showCreateTable("create table t(a varchar(10) binary charset utf8mb4 collate utf8mb4_unicode_ci) charset utf8 collate utf8_general_ci;") + c.Assert(strings.Contains(sct, "varchar(10) CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci"), IsTrue, Commentf(sct)) + c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci"), IsTrue, Commentf(sct)) + // case: (nil, nil, binary, nil) + sct = showCreateTable("create table t(a varchar(10) binary charset binary);") + c.Assert(strings.Contains(sct, "varbinary(10) DEFAULT NULL"), IsTrue, Commentf(sct)) + c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"), IsTrue, Commentf(sct)) +} + func (s *testIntegrationSuite) TestIssue15743(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -6990,6 +7041,18 @@ func (s *testIntegrationSerialSuite) TestIssue17891(c *C) { tk.MustExec("create table test(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_general_ci default 'a,B ,C');") } +func (s *testIntegrationSerialSuite) TestIssue20268(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `t` ( `a` enum('a','b') DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;") + tk.MustExec("insert into t values('a');") + tk.MustExec("select * from t where a = 'A';") +} + func (s *testIntegrationSerialSuite) TestIssue17233(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/scalar_function.go b/expression/scalar_function.go old mode 100755 new mode 100644 diff --git a/expression/scalar_function_test.go b/expression/scalar_function_test.go old mode 100755 new mode 100644 diff --git a/expression/testdata/flag_simplify_in.json b/expression/testdata/flag_simplify_in.json new file mode 100644 index 0000000000000..807a2e59984aa --- /dev/null +++ b/expression/testdata/flag_simplify_in.json @@ -0,0 +1,28 @@ +[ + { + "name": "TestSimplifyExpressionByFlag", + "cases": [ + "explain select * from t where a is null", + "explain select * from t where a is not null", + "explain select * from t where a > -1", + "explain select * from t where a <= -1", + "explain select * from t where a < 0", + "explain select * from t where a >= 0", + "explain select * from t where a = -1", + "explain select * from t where a <=> -1", + "explain select * from t where a != -1", + "explain select * from t where 0 > a", + "explain select * from t where 0 <= a", + "explain select * from t where -1 < a", + "explain select * from t where -1 >= a", + "explain select * from t where -1 = a", + "explain select * from t where -1 <=> a", + "explain select * from t where -1 != a", + // Tuples with null b should be filered out. + "explain select * from t where b >= 0", + "explain select * from t where b != -1", + // Int64 overflow corner case. + "explain select * from t where a = 0xFFFFFFFFFFFFFFFF" + ] + } +] diff --git a/expression/testdata/flag_simplify_out.json b/expression/testdata/flag_simplify_out.json new file mode 100644 index 0000000000000..62090584486ba --- /dev/null +++ b/expression/testdata/flag_simplify_out.json @@ -0,0 +1,134 @@ +[ + { + "Name": "TestSimplifyExpressionByFlag", + "Cases": [ + { + "SQL": "explain select * from t where a is null", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where a is not null", + "Plan": [ + "TableReader_6 10000.00 root data:TableFullScan_5", + "└─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where a > -1", + "Plan": [ + "TableReader_6 10000.00 root data:TableFullScan_5", + "└─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where a <= -1", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where a < 0", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where a >= 0", + "Plan": [ + "TableReader_6 10000.00 root data:TableFullScan_5", + "└─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where a = -1", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where a <=> -1", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where a != -1", + "Plan": [ + "TableReader_6 10000.00 root data:TableFullScan_5", + "└─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where 0 > a", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where 0 <= a", + "Plan": [ + "TableReader_6 10000.00 root data:TableFullScan_5", + "└─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where -1 < a", + "Plan": [ + "TableReader_6 10000.00 root data:TableFullScan_5", + "└─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where -1 >= a", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where -1 = a", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where -1 <=> a", + "Plan": [ + "TableDual_6 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t where -1 != a", + "Plan": [ + "TableReader_6 10000.00 root data:TableFullScan_5", + "└─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where b >= 0", + "Plan": [ + "TableReader_7 3333.33 root data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] ge(test.t.b, 0)", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where b != -1", + "Plan": [ + "TableReader_7 3333.33 root data:Selection_6", + "└─Selection_6 3333.33 cop[tikv] ne(test.t.b, -1)", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t where a = 0xFFFFFFFFFFFFFFFF", + "Plan": [ + "TableReader_7 10.00 root data:Selection_6", + "└─Selection_6 10.00 cop[tikv] eq(test.t.a, 18446744073709551615)", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + } +] diff --git a/expression/testdata/partition_pruner_out.json b/expression/testdata/partition_pruner_out.json index b3b34249c0108..02d188697d8a5 100644 --- a/expression/testdata/partition_pruner_out.json +++ b/expression/testdata/partition_pruner_out.json @@ -133,7 +133,7 @@ { "SQL": "explain select * from t5 where d is null", "Result": [ - "TableDual_5 0.00 root rows:0" + "TableDual_6 0.00 root rows:0" ] }, { diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 208eb9ca380ab..cc937c645014b 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -221,17 +221,17 @@ func (s *testInferTypeSuite) createTestCase4Columns() []typeInferTestCase { {"c_time_d ", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"c_timestamp ", mysql.TypeTimestamp, charset.CharsetBin, mysql.BinaryFlag, 24, 4}, {"c_timestamp_d", mysql.TypeTimestamp, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, - {"c_char ", mysql.TypeString, charset.CharsetUTF8MB4, 0, 20, 0}, // TODO: flag should be BinaryFlag - {"c_bchar ", mysql.TypeString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 20, 0}, - {"c_varchar ", mysql.TypeVarchar, charset.CharsetUTF8MB4, 0, 20, 0}, // TODO: BinaryFlag, tp should be TypeVarString - {"c_bvarchar ", mysql.TypeVarchar, charset.CharsetUTF8MB4, mysql.BinaryFlag, 20, 0}, // TODO: BinaryFlag, tp should be TypeVarString - {"c_text_d ", mysql.TypeBlob, charset.CharsetUTF8MB4, 0, 65535, 0}, // TODO: BlobFlag, BinaryFlag - {"c_btext_d ", mysql.TypeBlob, charset.CharsetUTF8MB4, mysql.BinaryFlag, 65535, 0}, // TODO: BlobFlag, BinaryFlag - {"c_binary ", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: BinaryFlag - {"c_varbinary ", mysql.TypeVarchar, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: BinaryFlag, tp should be TypeVarString - {"c_blob_d ", mysql.TypeBlob, charset.CharsetBin, mysql.BinaryFlag, 65535, 0}, // TODO: BlobFlag, BinaryFlag - {"c_set ", mysql.TypeSet, charset.CharsetUTF8MB4, 0, 5, 0}, // TODO: SetFlag, BinaryFlag - {"c_enum ", mysql.TypeEnum, charset.CharsetUTF8MB4, 0, 1, 0}, // TODO: EnumFlag, BinaryFlag + {"c_char ", mysql.TypeString, charset.CharsetUTF8MB4, 0, 20, 0}, + {"c_bchar ", mysql.TypeString, charset.CharsetUTF8MB4, 0, 20, 0}, + {"c_varchar ", mysql.TypeVarchar, charset.CharsetUTF8MB4, 0, 20, 0}, // TODO: tp should be TypeVarString + {"c_bvarchar ", mysql.TypeVarchar, charset.CharsetUTF8MB4, 0, 20, 0}, // TODO: tp should be TypeVarString + {"c_text_d ", mysql.TypeBlob, charset.CharsetUTF8MB4, 0, 65535, 0}, // TODO: BlobFlag + {"c_btext_d ", mysql.TypeBlob, charset.CharsetUTF8MB4, 0, 65535, 0}, // TODO: BlobFlag + {"c_binary ", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, + {"c_varbinary ", mysql.TypeVarchar, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: tp should be TypeVarString + {"c_blob_d ", mysql.TypeBlob, charset.CharsetBin, mysql.BinaryFlag, 65535, 0}, // TODO: BlobFlag + {"c_set ", mysql.TypeSet, charset.CharsetUTF8MB4, 0, 5, 0}, // TODO: SetFlag + {"c_enum ", mysql.TypeEnum, charset.CharsetUTF8MB4, 0, 1, 0}, // TODO: EnumFlag } } @@ -240,7 +240,7 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"strcmp(c_char, c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 2, 0}, {"space(c_int_d)", mysql.TypeLongBlob, mysql.DefaultCharset, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, {"CONCAT(c_binary, c_int_d)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 40, types.UnspecifiedLength}, - {"CONCAT(c_bchar, c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 40, types.UnspecifiedLength}, + {"CONCAT(c_bchar, c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 40, types.UnspecifiedLength}, {"CONCAT('T', 'i', 'DB')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 4, types.UnspecifiedLength}, {"CONCAT('T', 'i', 'DB', c_binary)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 24, types.UnspecifiedLength}, {"CONCAT_WS('-', 'T', 'i', 'DB')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 6, types.UnspecifiedLength}, @@ -1122,7 +1122,7 @@ func (s *testInferTypeSuite) createTestCase4Miscellaneous() []typeInferTestCase {"any_value(c_time_d)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"any_value(c_timestamp_d)", mysql.TypeTimestamp, charset.CharsetUTF8MB4, 0, 19, 0}, {"any_value(c_char)", mysql.TypeString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, - {"any_value(c_bchar)", mysql.TypeString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 20, types.UnspecifiedLength}, + {"any_value(c_bchar)", mysql.TypeString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"any_value(c_varchar)", mysql.TypeVarchar, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"any_value(c_text_d)", mysql.TypeBlob, charset.CharsetUTF8MB4, 0, 65535, types.UnspecifiedLength}, {"any_value(c_binary)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength}, diff --git a/go.mod b/go.mod index 74c870fed9007..fd896eade4824 100644 --- a/go.mod +++ b/go.mod @@ -23,7 +23,7 @@ require ( github.com/klauspost/cpuid v1.2.1 github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef - github.com/ngaut/unistore v0.0.0-20200918091209-68a2db6bb775 + github.com/ngaut/unistore v0.0.0-20200929093420-76a7b18be28e github.com/opentracing/basictracer-go v1.0.0 github.com/opentracing/opentracing-go v1.1.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 @@ -36,7 +36,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20200828054126-d677e6fd224a github.com/pingcap/log v0.0.0-20200828042413-fce0951f1463 - github.com/pingcap/parser v0.0.0-20200929032957-9678b2b7cefb + github.com/pingcap/parser v0.0.0-20201014065945-fb6bde872a79 github.com/pingcap/sysutil v0.0.0-20200715082929-4c47bcac246a github.com/pingcap/tidb-tools v4.0.5-0.20200820092506-34ea90c93237+incompatible github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3 @@ -63,6 +63,7 @@ require ( golang.org/x/tools v0.0.0-20200820010801-b793a1359eac google.golang.org/grpc v1.26.0 gopkg.in/natefinch/lumberjack.v2 v2.0.0 + honnef.co/go/tools v0.0.1-2020.1.6 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) diff --git a/go.sum b/go.sum index 64d2e1dcd7886..e250cefda157b 100644 --- a/go.sum +++ b/go.sum @@ -401,8 +401,8 @@ github.com/ngaut/unistore v0.0.0-20200803051709-607d96233b1d/go.mod h1:2QAH8tXCj github.com/ngaut/unistore v0.0.0-20200806113332-5b9f73333a19/go.mod h1:RtZJKyiaHRiII+b9/g/4339rSikSvfrUJmIbrUkYVi4= github.com/ngaut/unistore v0.0.0-20200820080223-c734bcc4ea53/go.mod h1:85S5ZgzoHtTMyaEYhaWnxv9OWMBfyhTNuWypXCfVn/0= github.com/ngaut/unistore v0.0.0-20200828072424-1c0ede06a3fc/go.mod h1:iSlx5Ub/926GvQn6+d2B2C16wJJwgQIsi6k/bEU0vl4= -github.com/ngaut/unistore v0.0.0-20200918091209-68a2db6bb775 h1:9z7ZQ//+b6DjI79ak0sNf7RMgmjYh/BTB+NkipfdLiM= -github.com/ngaut/unistore v0.0.0-20200918091209-68a2db6bb775/go.mod h1:ZR3NH+HzqfiYetwdoAivApnIy8iefPZHTMLfrFNm8g4= +github.com/ngaut/unistore v0.0.0-20200929093420-76a7b18be28e h1:1a8YbLM1sBmwEjzEVT/JD12Vjf6BNnBBEUV3nAcZYKU= +github.com/ngaut/unistore v0.0.0-20200929093420-76a7b18be28e/go.mod h1:ZR3NH+HzqfiYetwdoAivApnIy8iefPZHTMLfrFNm8g4= github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= @@ -507,8 +507,8 @@ github.com/pingcap/parser v0.0.0-20200730092557-34a468e9b774/go.mod h1:vQdbJqobJ github.com/pingcap/parser v0.0.0-20200731033026-84f62115187c/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/parser v0.0.0-20200813083329-a4bff035d3e2/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/parser v0.0.0-20200821073936-cf85e80665c4/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200929032957-9678b2b7cefb h1:Nlswd41UZDaedHNysE/lb8dc3EpmWAApf480qU2N3nU= -github.com/pingcap/parser v0.0.0-20200929032957-9678b2b7cefb/go.mod h1:RlLfMRJwFBSiXd2lUaWdV5pSXtrpyvZM8k5bbZWsheU= +github.com/pingcap/parser v0.0.0-20201014065945-fb6bde872a79 h1:Dcxi/lDJ6C3M5ocRbhR66MBDMmqFkPVt/Y79DVb5QR8= +github.com/pingcap/parser v0.0.0-20201014065945-fb6bde872a79/go.mod h1:RlLfMRJwFBSiXd2lUaWdV5pSXtrpyvZM8k5bbZWsheU= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181/go.mod h1:q4HTx/bA8aKBa4S7L+SQKHvjRPXCRV0tA0yRw0qkZSA= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200714122454-1a64f969cb3c/go.mod h1:v/dY4mVkt3dh/Liphhk0E4ScOkZpIk0m0GvWJ9FapDs= @@ -889,6 +889,7 @@ golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200325010219-a49f79bcc224/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200410194907-79a7a3126eef/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200521211927-2b542361a4fc/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200820010801-b793a1359eac h1:DugppSxw0LSF8lcjaODPJZoDzq0ElTGskTst3ZaBkHI= @@ -980,6 +981,8 @@ honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9 honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.0.1-2020.1.5 h1:nI5egYTGJakVyOryqLs1cQO5dO0ksin5XXs2pspk75k= honnef.co/go/tools v0.0.1-2020.1.5/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.6 h1:W18jzjh8mfPez+AwGLxmOImucz/IFjpNlrKVnaj2YVc= +honnef.co/go/tools v0.0.1-2020.1.6/go.mod h1:pyyisuGw24ruLjrr1ddx39WE0y9OooInRzEYLhQB2YY= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= diff --git a/hooks/pre-commit b/hooks/pre-commit index 065abcf0d6ddd..c10007e65031a 100755 --- a/hooks/pre-commit +++ b/hooks/pre-commit @@ -5,7 +5,7 @@ # The PR is https://github.com/kubernetes/kubernetes/pull/47673 # How to use this hook? -# ln -s hooks/pre-commit .git/hooks/ +# ln -s ../../hooks/pre-commit .git/hooks/pre-commit # In case hook is not executable # chmod +x .git/hooks/pre-commit diff --git a/infoschema/builder.go b/infoschema/builder.go index 146dd6298b764..deff4979a8b21 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -83,7 +83,9 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro // For normal node's information schema, repaired table is existed. // For repair node's information schema, repaired table is filtered (couldn't find it in `is`). // So here skip to reserve the allocators when repairing table. - diff.Type != model.ActionRepairTable { + diff.Type != model.ActionRepairTable && + // Alter sequence will change the sequence info in the allocator, so the old allocator is not valid any more. + diff.Type != model.ActionAlterSequence { oldAllocs, _ := b.is.AllocByID(oldTableID) allocs = filterAllocators(diff, oldAllocs) } diff --git a/infoschema/metric_table_def.go b/infoschema/metric_table_def.go index f74820d6f1055..82146bb87ef5b 100644 --- a/infoschema/metric_table_def.go +++ b/infoschema/metric_table_def.go @@ -605,8 +605,8 @@ var MetricTableMap = map[string]MetricTableDef{ }, "tidb_gc_duration": { Comment: "The quantile of kv storage garbage collection time durations", - PromQL: "histogram_quantile($QUANTILE, sum(rate(tidb_tikvclient_gc_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance))", - Labels: []string{"instance"}, + PromQL: "histogram_quantile($QUANTILE, sum(rate(tidb_tikvclient_gc_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,instance,stage))", + Labels: []string{"instance", "stage"}, Quantile: 0.95, }, "tidb_gc_config": { @@ -2522,13 +2522,13 @@ var MetricTableMap = map[string]MetricTableDef{ Comment: "The total time of kv storage range worker processing one task duration", }, "tidb_gc_total_count": { - PromQL: "sum(increase(tidb_tikvclient_gc_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", - Labels: []string{"instance"}, + PromQL: "sum(increase(tidb_tikvclient_gc_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,stage)", + Labels: []string{"instance", "stage"}, Comment: "The total count of kv storage garbage collection", }, "tidb_gc_total_time": { - PromQL: "sum(increase(tidb_tikvclient_gc_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", - Labels: []string{"instance"}, + PromQL: "sum(increase(tidb_tikvclient_gc_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,stage)", + Labels: []string{"instance", "stage"}, Comment: "The total time of kv storage garbage collection time durations", }, "tidb_get_token_total_count": { diff --git a/infoschema/tables.go b/infoschema/tables.go old mode 100755 new mode 100644 diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go old mode 100755 new mode 100644 diff --git a/meta/meta.go b/meta/meta.go index 539a39f1cb730..7d559d11d6ff1 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -381,6 +381,22 @@ func (m *Meta) CreateSequenceAndSetSeqValue(dbID int64, tableInfo *model.TableIn return errors.Trace(err) } +// RestartSequenceValue resets the the sequence value. +func (m *Meta) RestartSequenceValue(dbID int64, tableInfo *model.TableInfo, seqValue int64) error { + // Check if db exists. + dbKey := m.dbKey(dbID) + if err := m.checkDBExists(dbKey); err != nil { + return errors.Trace(err) + } + + // Check if table exists. + tableKey := m.tableKey(tableInfo.ID) + if err := m.checkTableExists(dbKey, tableKey); err != nil { + return errors.Trace(err) + } + return errors.Trace(m.txn.HSet(m.dbKey(dbID), m.sequenceKey(tableInfo.ID), []byte(strconv.FormatInt(seqValue, 10)))) +} + // DropDatabase drops whole database. func (m *Meta) DropDatabase(dbID int64) error { // Check if db exists. diff --git a/metrics/session.go b/metrics/session.go index 4838ee771ac8b..feb38783ce161 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -142,4 +142,5 @@ const ( LblAddress = "address" LblBatchGet = "batch_get" LblGet = "get" + LblLockKeys = "lock_keys" ) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 5ee932438bed0..ee857e8e44881 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -928,3 +928,35 @@ func (s *testAnalyzeSuite) TestIndexEqualUnknown(c *C) { testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testAnalyzeSuite) TestLimitIndexEstimation(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, key idx_a(a), key idx_b(b))") + // Values in column a are from 1 to 1000000, values in column b are from 1000000 to 1, + // these 2 columns are strictly correlated in reverse order. + err = s.loadTableStats("analyzeSuiteTestLimitIndexEstimationT.json", dom) + c.Assert(err, IsNil) + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b6db7043b9c4b..3b67e8f66fd78 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" @@ -622,7 +623,10 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou func (p *LogicalJoin) getIndexJoinBuildHelper(ds *DataSource, innerJoinKeys []*expression.Column, checkPathValid func(path *util.AccessPath) bool) (*indexJoinBuildHelper, []int) { - helper := &indexJoinBuildHelper{join: p} + helper := &indexJoinBuildHelper{ + join: p, + innerPlan: ds, + } for _, path := range ds.possibleAccessPaths { if checkPathValid(path) { emptyRange, err := helper.analyzeLookUpFilters(path, ds, innerJoinKeys) @@ -741,7 +745,7 @@ func (p *LogicalJoin) buildIndexJoinInner2IndexScan( joins = make([]PhysicalPlan, 0, 3) rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.IdxCols, outerJoinKeys) maxOneRow := false - if helper.chosenPath.Index.Unique && helper.maxUsedCols == len(helper.chosenPath.FullIdxCols) { + if helper.chosenPath.Index.Unique && helper.usedColsLen == len(helper.chosenPath.FullIdxCols) { l := len(helper.chosenAccess) if l == 0 { maxOneRow = true @@ -774,10 +778,12 @@ func (p *LogicalJoin) buildIndexJoinInner2IndexScan( } type indexJoinBuildHelper struct { - join *LogicalJoin + join *LogicalJoin + innerPlan *DataSource chosenIndexInfo *model.IndexInfo - maxUsedCols int + usedColsLen int + usedColsNDV float64 chosenAccess []expression.Expression chosenRemained []expression.Expression idxOff2KeyOff []int @@ -1272,7 +1278,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath if emptyRange { return true, nil } - ijHelper.updateBestChoice(ranges, path, accesses, remained, nil) + ijHelper.updateBestChoice(ranges, path, accesses, remained, nil, lastColPos) return false, nil } lastPossibleCol := path.IdxCols[lastColPos] @@ -1311,7 +1317,10 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath remained = append(remained, colAccesses...) } accesses = append(accesses, colAccesses...) - ijHelper.updateBestChoice(ranges, path, accesses, remained, nil) + if len(colAccesses) > 0 { + lastColPos = lastColPos + 1 + } + ijHelper.updateBestChoice(ranges, path, accesses, remained, nil, lastColPos) return false, nil } accesses = append(accesses, lastColAccess...) @@ -1323,24 +1332,38 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath if emptyRange { return true, nil } - ijHelper.updateBestChoice(ranges, path, accesses, remained, lastColManager) + ijHelper.updateBestChoice(ranges, path, accesses, remained, lastColManager, lastColPos+1) return false, nil } func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, path *util.AccessPath, accesses, - remained []expression.Expression, lastColManager *ColWithCmpFuncManager) { - // We choose the index by the number of used columns of the range, the much the better. - // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. - // But obviously when the range is nil, we don't need index join. - if len(ranges) > 0 && len(ranges[0].LowVal) > ijHelper.maxUsedCols { - ijHelper.chosenPath = path - ijHelper.maxUsedCols = len(ranges[0].LowVal) - ijHelper.chosenRanges = ranges - ijHelper.chosenAccess = accesses - ijHelper.chosenRemained = remained - ijHelper.idxOff2KeyOff = ijHelper.curIdxOff2KeyOff - ijHelper.lastColManager = lastColManager + remained []expression.Expression, lastColManager *ColWithCmpFuncManager, usedColsLen int) { + // Notice that there may be the cases like `t1.a = t2.a and b > 2 and b < 1`, so ranges can be nil though the conditions are valid. + // Obviously when the range is nil, we don't need index join. + if len(ranges) == 0 { + return + } + var innerNDV float64 + if stats := ijHelper.innerPlan.statsInfo(); stats != nil && stats.StatsVersion != statistics.PseudoVersion { + innerNDV = getCardinality(path.IdxCols[:usedColsLen], ijHelper.innerPlan.Schema(), stats) + } + // We choose the index by the NDV of the used columns, the larger the better. + // If NDVs are same, we choose index which uses more columns. + // Note that these 2 heuristic rules are too simple to cover all cases, + // since the NDV of outer join keys are not considered, and the detached access conditions + // may contain expressions like `t1.a > t2.a`. It's pretty hard to evaluate the join selectivity + // of these non-column-equal conditions, so I prefer to keep these heuristic rules simple at least for now. + if innerNDV < ijHelper.usedColsNDV || (innerNDV == ijHelper.usedColsNDV && usedColsLen <= ijHelper.usedColsLen) { + return } + ijHelper.chosenPath = path + ijHelper.usedColsLen = len(ranges[0].LowVal) + ijHelper.usedColsNDV = innerNDV + ijHelper.chosenRanges = ranges + ijHelper.chosenAccess = accesses + ijHelper.chosenRemained = remained + ijHelper.idxOff2KeyOff = ijHelper.curIdxOff2KeyOff + ijHelper.lastColManager = lastColManager } func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, emptyRange bool, err error) { @@ -1879,7 +1902,7 @@ func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalPrope childProp := &property.PhysicalProperty{ ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.stats.RowCount, prop.ExpectedCnt), Enforced: true, - Items: property.ItemsFromCols(la.groupByCols, desc), + Items: property.ItemsFromCols(la.GetGroupByCols(), desc), } if !prop.IsPrefix(childProp) { return enforcedAggs @@ -1938,7 +1961,8 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P } } // group by a + b is not interested in any order. - if len(la.groupByCols) != len(la.GroupByItems) { + groupByCols := la.GetGroupByCols() + if len(groupByCols) != len(la.GroupByItems) { return nil } @@ -1949,7 +1973,7 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P } for _, possibleChildProperty := range la.possibleProperties { - childProp.Items = property.ItemsFromCols(possibleChildProperty[:len(la.groupByCols)], desc) + childProp.Items = property.ItemsFromCols(possibleChildProperty[:len(groupByCols)], desc) if !prop.IsPrefix(childProp) { continue } diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index c3569527df786..df5a83dffb1f1 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -21,7 +21,9 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" ) @@ -82,6 +84,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { DBName: model.NewCIStr("test"), }) dataSourceNode.schema = dsSchema + dataSourceNode.stats = &property.StatsInfo{StatsVersion: statistics.PseudoVersion} outerChildSchema := expression.NewSchema() var outerChildNames types.NameSlice outerChildSchema.Append(&expression.Column{ @@ -244,7 +247,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { others, err := s.rewriteSimpleExpr(tt.otherConds, joinNode.schema, joinColNames) c.Assert(err, IsNil) joinNode.OtherConditions = others - helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil} + helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil, innerPlan: dataSourceNode} _, err = helper.analyzeLookUpFilters(path, dataSourceNode, tt.innerKeys) c.Assert(err, IsNil) c.Assert(fmt.Sprintf("%v", helper.chosenAccess), Equals, tt.accesses) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index ec5e5192cfbfd..c6a2ae423e8cc 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1182,9 +1182,9 @@ func (ds *DataSource) splitIndexFilterConditions(conditions []expression.Express return indexConditions, tableConditions } -// getMostCorrColFromExprs checks if column in the condition is correlated enough with handle. If the condition +// getMostCorrCol4Handle checks if column in the condition is correlated enough with handle. If the condition // contains multiple columns, return nil and get the max correlation, which would be used in the heuristic estimation. -func getMostCorrColFromExprs(exprs []expression.Expression, histColl *statistics.Table, threshold float64) (*expression.Column, float64) { +func getMostCorrCol4Handle(exprs []expression.Expression, histColl *statistics.Table, threshold float64) (*expression.Column, float64) { var cols []*expression.Column cols = expression.ExtractColumnsFromExpressions(cols, exprs, nil) if len(cols) == 0 { @@ -1202,13 +1202,13 @@ func getMostCorrColFromExprs(exprs []expression.Expression, histColl *statistics if !ok { continue } - curCorr := math.Abs(hist.Correlation) - if corrCol == nil || corr < curCorr { + curCorr := hist.Correlation + if corrCol == nil || math.Abs(corr) < math.Abs(curCorr) { corrCol = col corr = curCorr } } - if len(colSet) == 1 && corr >= threshold { + if len(colSet) == 1 && math.Abs(corr) >= threshold { return corrCol, corr } return nil, corr @@ -1273,28 +1273,31 @@ func convertRangeFromExpectedCnt(ranges []*ranger.Range, rangeCounts []float64, return convertedRanges, count, false } -// crossEstimateRowCount estimates row count of table scan using histogram of another column which is in TableFilters +// crossEstimateTableRowCount estimates row count of table scan using histogram of another column which is in TableFilters // and has high order correlation with handle column. For example, if the query is like: // `select * from tbl where a = 1 order by pk limit 1` // if order of column `a` is strictly correlated with column `pk`, the row count of table scan should be: // `1 + row_count(a < 1 or a is null)` -func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { +func (ds *DataSource) crossEstimateTableRowCount(path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { if ds.statisticTable.Pseudo || len(path.TableFilters) == 0 { return 0, false, 0 } - col, corr := getMostCorrColFromExprs(path.TableFilters, ds.statisticTable, ds.ctx.GetSessionVars().CorrelationThreshold) - // If table scan is not full range scan, we cannot use histogram of other columns for estimation, because + col, corr := getMostCorrCol4Handle(path.TableFilters, ds.statisticTable, ds.ctx.GetSessionVars().CorrelationThreshold) + return ds.crossEstimateRowCount(path, path.TableFilters, col, corr, expectedCnt, desc) +} + +// crossEstimateRowCount is the common logic of crossEstimateTableRowCount and crossEstimateIndexRowCount. +func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expression.Expression, col *expression.Column, corr, expectedCnt float64, desc bool) (float64, bool, float64) { + // If the scan is not full range scan, we cannot use histogram of other columns for estimation, because // the histogram reflects value distribution in the whole table level. if col == nil || len(path.AccessConds) > 0 { return 0, false, corr } - colInfoID := col.ID - colID := col.UniqueID - colHist := ds.statisticTable.Columns[colInfoID] - if colHist.Correlation < 0 { + colInfoID, colID := col.ID, col.UniqueID + if corr < 0 { desc = !desc } - accessConds, remained := ranger.DetachCondsForColumn(ds.ctx, path.TableFilters, col) + accessConds, remained := ranger.DetachCondsForColumn(ds.ctx, conds, col) if len(accessConds) == 0 { return 0, false, corr } @@ -1332,6 +1335,62 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, expectedCnt f return scanCount, true, 0 } +// crossEstimateIndexRowCount estimates row count of index scan using histogram of another column which is in TableFilters/IndexFilters +// and has high order correlation with the first index column. For example, if the query is like: +// `select * from tbl where a = 1 order by b limit 1` +// if order of column `a` is strictly correlated with column `b`, the row count of IndexScan(b) should be: +// `1 + row_count(a < 1 or a is null)` +func (ds *DataSource) crossEstimateIndexRowCount(path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { + filtersLen := len(path.TableFilters) + len(path.IndexFilters) + if ds.statisticTable.Pseudo || filtersLen == 0 { + return 0, false, 0 + } + col, corr := getMostCorrCol4Index(path, ds.statisticTable, ds.ctx.GetSessionVars().CorrelationThreshold) + filters := make([]expression.Expression, 0, filtersLen) + filters = append(filters, path.TableFilters...) + filters = append(filters, path.IndexFilters...) + return ds.crossEstimateRowCount(path, filters, col, corr, expectedCnt, desc) +} + +// getMostCorrCol4Index checks if column in the condition is correlated enough with the first index column. If the condition +// contains multiple columns, return nil and get the max correlation, which would be used in the heuristic estimation. +func getMostCorrCol4Index(path *util.AccessPath, histColl *statistics.Table, threshold float64) (*expression.Column, float64) { + if histColl.ExtendedStats == nil || len(histColl.ExtendedStats.Stats) == 0 { + return nil, 0 + } + var cols []*expression.Column + cols = expression.ExtractColumnsFromExpressions(cols, path.TableFilters, nil) + cols = expression.ExtractColumnsFromExpressions(cols, path.IndexFilters, nil) + if len(cols) == 0 { + return nil, 0 + } + colSet := set.NewInt64Set() + var corr float64 + var corrCol *expression.Column + for _, col := range cols { + if colSet.Exist(col.UniqueID) { + continue + } + colSet.Insert(col.UniqueID) + curCorr := float64(0) + for _, item := range histColl.ExtendedStats.Stats { + if (col.ID == item.ColIDs[0] && path.FullIdxCols[0].ID == item.ColIDs[1]) || + (col.ID == item.ColIDs[1] && path.FullIdxCols[0].ID == item.ColIDs[0]) { + curCorr = item.ScalarVals + break + } + } + if corrCol == nil || math.Abs(corr) < math.Abs(curCorr) { + corrCol = col + corr = curCorr + } + } + if len(colSet) == 1 && math.Abs(corr) >= threshold { + return corrCol, corr + } + return nil, corr +} + // GetPhysicalScan returns PhysicalTableScan for the LogicalTableScan. func (s *LogicalTableScan) GetPhysicalScan(schema *expression.Schema, stats *property.StatsInfo) *PhysicalTableScan { ds := s.Source @@ -1608,7 +1667,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper } rowCount := path.CountAfterAccess if prop.ExpectedCnt < ds.stats.RowCount { - count, ok, corr := ds.crossEstimateRowCount(path, prop.ExpectedCnt, isMatchProp && prop.Items[0].Desc) + count, ok, corr := ds.crossEstimateTableRowCount(path, prop.ExpectedCnt, isMatchProp && prop.Items[0].Desc) if ok { // TODO: actually, before using this count as the estimated row count of table scan, we need additionally // check if count < row_count(first_region | last_region), and use the larger one since we build one copTask @@ -1618,8 +1677,8 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper // Considering that when this scenario happens, the execution time is close between IndexScan and TableScan, // we do not add this check temporarily. rowCount = count - } else if corr < 1 { - correlationFactor := math.Pow(1-corr, float64(ds.ctx.GetSessionVars().CorrelationExpFactor)) + } else if abs := math.Abs(corr); abs < 1 { + correlationFactor := math.Pow(1-abs, float64(ds.ctx.GetSessionVars().CorrelationExpFactor)) selectivity := ds.stats.RowCount / rowCount rowCount = math.Min(prop.ExpectedCnt/selectivity/correlationFactor, rowCount) } @@ -1681,12 +1740,15 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper } rowCount := path.CountAfterAccess is.initSchema(append(path.FullIdxCols, ds.commonHandleCols...), !isSingleScan) - // Only use expectedCnt when it's smaller than the count we calculated. - // e.g. IndexScan(count1)->After Filter(count2). The `ds.stats.RowCount` is count2. count1 is the one we need to calculate - // If expectedCnt and count2 are both zero and we go into the below `if` block, the count1 will be set to zero though it's shouldn't be. if (isMatchProp || prop.IsEmpty()) && prop.ExpectedCnt < ds.stats.RowCount { - selectivity := ds.stats.RowCount / path.CountAfterAccess - rowCount = math.Min(prop.ExpectedCnt/selectivity, rowCount) + count, ok, corr := ds.crossEstimateIndexRowCount(path, prop.ExpectedCnt, isMatchProp && prop.Items[0].Desc) + if ok { + rowCount = count + } else if abs := math.Abs(corr); abs < 1 { + correlationFactor := math.Pow(1-abs, float64(ds.ctx.GetSessionVars().CorrelationExpFactor)) + selectivity := ds.stats.RowCount / rowCount + rowCount = math.Min(prop.ExpectedCnt/selectivity/correlationFactor, rowCount) + } } is.stats = ds.tableStats.ScaleByExpectCnt(rowCount) rowSize := is.indexScanRowSize(idx, ds, true) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 36b23dc105dd3..6289bf76933d8 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1198,6 +1198,31 @@ func (s *testIntegrationSuite) TestTableDualWithRequiredProperty(c *C) { tk.MustExec("select /*+ MERGE_JOIN(t1, t2) */ * from t1 partition (p0), t2 where t1.a > 100 and t1.a = t2.a") } +func (s *testIntegrationSuite) TestIndexJoinInnerIndexNDV(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int not null, b int not null, c int not null)") + tk.MustExec("create table t2(a int not null, b int not null, c int not null, index idx1(a,b), index idx2(c))") + tk.MustExec("insert into t1 values(1,1,1),(1,1,1),(1,1,1)") + tk.MustExec("insert into t2 values(1,1,1),(1,1,2),(1,1,3)") + tk.MustExec("analyze table t1, t2") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } +} + func (s *testIntegrationSerialSuite) TestIssue16837(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1525,6 +1550,28 @@ func (s *testIntegrationSerialSuite) TestExplainAnalyzePointGet(c *C) { checkExplain("BatchGet") } +func (s *testIntegrationSerialSuite) TestExplainAnalyzeDML(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(" create table t (a int, b int, unique index (a));") + tk.MustExec("insert into t values (1,1)") + + res := tk.MustQuery("explain analyze select * from t where a=1;") + checkExplain := func(rpc string) { + resBuff := bytes.NewBufferString("") + for _, row := range res.Rows() { + fmt.Fprintf(resBuff, "%s\n", row) + } + explain := resBuff.String() + c.Assert(strings.Contains(explain, rpc+":{num_rpc:"), IsTrue, Commentf("%s", explain)) + c.Assert(strings.Contains(explain, "total_time:"), IsTrue, Commentf("%s", explain)) + } + checkExplain("Get") + res = tk.MustQuery("explain analyze insert ignore into t values (1,1),(2,2),(3,3),(4,4);") + checkExplain("BatchGet") +} + func (s *testIntegrationSuite) TestPartitionExplain(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 0b2d79840e2f9..267a789b8bd85 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -113,15 +113,6 @@ const ( ErrExprInOrderBy = "ORDER BY" ) -func (la *LogicalAggregation) collectGroupByColumns() { - la.groupByCols = la.groupByCols[:0] - for _, item := range la.GroupByItems { - if col, ok := item.(*expression.Column); ok { - la.groupByCols = append(la.groupByCols, col) - } - } -} - // aggOrderByResolver is currently resolving expressions of order by clause // in aggregate function GROUP_CONCAT. type aggOrderByResolver struct { @@ -260,7 +251,6 @@ func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFu plan4Agg.SetChildren(p) plan4Agg.GroupByItems = gbyItems plan4Agg.SetSchema(schema4Agg) - plan4Agg.collectGroupByColumns() return plan4Agg, aggIndexMap, nil } @@ -1083,7 +1073,6 @@ func (b *PlanBuilder) buildDistinct(child LogicalPlan, length int) (*LogicalAggr if hint := b.TableHints(); hint != nil { plan4Agg.aggHints = hint.aggHints } - plan4Agg.collectGroupByColumns() for _, col := range child.Schema().Columns { aggDesc, err := aggregation.NewAggFuncDesc(b.ctx, ast.AggFuncFirstRow, []expression.Expression{col}, false) if err != nil { @@ -2857,10 +2846,10 @@ func getStatsTable(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) } var statsTbl *statistics.Table - if pid != tblInfo.ID { - statsTbl = statsHandle.GetPartitionStats(tblInfo, pid) - } else { + if pid == tblInfo.ID || ctx.GetSessionVars().UseDynamicPartitionPrune() { statsTbl = statsHandle.GetTableStats(tblInfo) + } else { + statsTbl = statsHandle.GetPartitionStats(tblInfo, pid) } // 2. table row count from statistics is zero. @@ -2976,7 +2965,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as columns = tbl.Cols() } var statisticTable *statistics.Table - if _, ok := tbl.(table.PartitionedTable); !ok { + if _, ok := tbl.(table.PartitionedTable); !ok || b.ctx.GetSessionVars().UseDynamicPartitionPrune() { statisticTable = getStatsTable(b.ctx, tbl.Meta(), tbl.Meta().ID) } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index ce26c680af33f..704af732ea96d 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -553,6 +553,31 @@ func (s *testPlanSuite) TestColumnPruning(c *C) { } } +func (s *testPlanSuite) TestSortByItemsPruning(c *C) { + defer testleak.AfterTest(c)() + var ( + input []string + output [][]string + ) + s.testData.GetTestCases(c, &input, &output) + s.testData.OnRecord(func() { + output = make([][]string, len(input)) + }) + + ctx := context.Background() + for i, tt := range input { + comment := Commentf("for %s", tt) + stmt, err := s.ParseOneStmt(tt, "", "") + c.Assert(err, IsNil, comment) + + p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + c.Assert(err, IsNil) + lp, err := logicalOptimize(ctx, flagEliminateProjection|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) + c.Assert(err, IsNil) + s.checkOrderByItems(lp, c, &output[i], comment) + } +} + func (s *testPlanSuite) TestProjectionEliminator(c *C) { defer testleak.AfterTest(c)() tests := []struct { @@ -620,6 +645,27 @@ func (s *testPlanSuite) checkDataSourceCols(p LogicalPlan, c *C, ans map[int][]s } } +func (s *testPlanSuite) checkOrderByItems(p LogicalPlan, c *C, colList *[]string, comment CommentInterface) { + switch p := p.(type) { + case *LogicalSort: + s.testData.OnRecord(func() { + *colList = make([]string, len(p.ByItems)) + }) + for i, col := range p.ByItems { + s.testData.OnRecord(func() { + (*colList)[i] = col.String() + }) + s := col.String() + c.Assert(s, Equals, (*colList)[i], comment) + } + } + children := p.Children() + c.Assert(len(children), LessEqual, 1, Commentf("For %v Expected <= 1 Child", comment)) + for _, child := range children { + s.checkOrderByItems(child, c, colList, comment) + } +} + func (s *testPlanSuite) TestValidate(c *C) { defer testleak.AfterTest(c)() tests := []struct { diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 83e74590d6c70..9f26f1f211924 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -307,8 +307,6 @@ type LogicalAggregation struct { AggFuncs []*aggregation.AggFuncDesc GroupByItems []expression.Expression - // groupByCols stores the columns that are group-by items. - groupByCols []*expression.Column // aggHints stores aggregation hint information. aggHints aggHintInfo @@ -349,14 +347,16 @@ func (la *LogicalAggregation) IsCompleteModeAgg() bool { return la.AggFuncs[0].Mode == aggregation.CompleteMode } -// GetGroupByCols returns the groupByCols. If the groupByCols haven't be collected, -// this method would collect them at first. If the GroupByItems have been changed, -// we should explicitly collect GroupByColumns before this method. +// GetGroupByCols returns the columns that are group-by items. +// For example, `group by a, b, c+d` will return [a, b]. func (la *LogicalAggregation) GetGroupByCols() []*expression.Column { - if la.groupByCols == nil { - la.collectGroupByColumns() + groupByCols := make([]*expression.Column, 0, len(la.GroupByItems)) + for _, item := range la.GroupByItems { + if col, ok := item.(*expression.Column); ok { + groupByCols = append(groupByCols, col) + } } - return la.groupByCols + return groupByCols } // ExtractCorrelatedCols implements LogicalPlan interface. diff --git a/planner/core/mock.go b/planner/core/mock.go index ba7c1145ccc38..8eee3c8647b68 100644 --- a/planner/core/mock.go +++ b/planner/core/mock.go @@ -14,6 +14,8 @@ package core import ( + "fmt" + "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -370,7 +372,9 @@ func MockContext() sessionctx.Context { } ctx.GetSessionVars().CurrentDB = "test" do := &domain.Domain{} - do.CreateStatsHandle(ctx) + if err := do.CreateStatsHandle(ctx); err != nil { + panic(fmt.Sprintf("create mock context panic: %+v", err)) + } domain.BindDomain(ctx, do) return ctx } diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 5e6a2b948db1b..97ec3dc8dd884 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -284,7 +284,7 @@ func SetPBColumnsDefaultValue(ctx sessionctx.Context, pbColumns []*tipb.ColumnIn if c.IsGenerated() && !c.GeneratedStored { pbColumns[i].DefaultVal = []byte{codec.NilFlag} } - if c.OriginDefaultValue == nil { + if c.GetOriginDefaultValue() == nil { continue } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 3b77a3daaa02f..9873208a81bd1 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3001,6 +3001,13 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err v.Table.Name.L, "", authErr) } } + case *ast.AlterSequenceStmt: + if b.ctx.GetSessionVars().User != nil { + authErr = ErrTableaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername, + b.ctx.GetSessionVars().User.AuthHostname, v.Name.Name.L) + } + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.Name.Schema.L, + v.Name.Name.L, "", authErr) case *ast.CreateDatabaseStmt: if b.ctx.GetSessionVars().User != nil { authErr = ErrDBaccessDenied.GenWithStackByArgs(b.ctx.GetSessionVars().User.AuthUsername, diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index fa89750f81ffc..05a8c9890cbfc 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -91,7 +91,7 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { "└─Point_Get_1 1.00 root table:t handle:1", )) tk.MustQuery("explain select a from t where a = -1").Check(testkit.Rows( - "TableDual_5 0.00 root rows:0", + "TableDual_6 0.00 root rows:0", )) tk.MustExec(`prepare stmt0 from "select a from t where a = ?"`) tk.MustExec("set @p0 = -1") diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index a8b4d1d47a5b1..eb5cbad22d13e 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -188,6 +188,11 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { if node.Kind == ast.BRIEKindRestore { p.flag |= inCreateOrDropTable } + case *ast.TableSource: + isModeOracle := p.ctx.GetSessionVars().SQLMode&mysql.ModeOracle != 0 + if _, ok := node.Source.(*ast.SelectStmt); ok && !isModeOracle && len(node.AsName.L) == 0 { + p.err = ddl.ErrDerivedMustHaveAlias.GenWithStackByArgs() + } case *ast.CreateStatisticsStmt, *ast.DropStatisticsStmt: p.checkStatisticsOpGrammar(in) default: @@ -580,13 +585,16 @@ func (p *preprocessor) checkNonUniqTableAlias(stmt *ast.Join) { p.tableAliasInJoin = append(p.tableAliasInJoin, make(map[string]interface{})) } tableAliases := p.tableAliasInJoin[len(p.tableAliasInJoin)-1] - if err := isTableAliasDuplicate(stmt.Left, tableAliases); err != nil { - p.err = err - return - } - if err := isTableAliasDuplicate(stmt.Right, tableAliases); err != nil { - p.err = err - return + isOracleMode := p.ctx.GetSessionVars().SQLMode&mysql.ModeOracle != 0 + if !isOracleMode { + if err := isTableAliasDuplicate(stmt.Left, tableAliases); err != nil { + p.err = err + return + } + if err := isTableAliasDuplicate(stmt.Right, tableAliases); err != nil { + p.err = err + return + } } p.flag |= parentIsJoin } diff --git a/planner/core/property_cols_prune.go b/planner/core/property_cols_prune.go index a57a06993893e..c8296aa799283 100644 --- a/planner/core/property_cols_prune.go +++ b/planner/core/property_cols_prune.go @@ -200,10 +200,11 @@ func (la *LogicalAggregation) PreparePossibleProperties(schema *expression.Schem return nil } resultProperties := make([][]*expression.Column, 0, len(childProps)) + groupByCols := la.GetGroupByCols() for _, possibleChildProperty := range childProps { - sortColOffsets := getMaxSortPrefix(possibleChildProperty, la.groupByCols) - if len(sortColOffsets) == len(la.groupByCols) { - resultProperties = append(resultProperties, possibleChildProperty[:len(la.groupByCols)]) + sortColOffsets := getMaxSortPrefix(possibleChildProperty, groupByCols) + if len(sortColOffsets) == len(groupByCols) { + resultProperties = append(resultProperties, possibleChildProperty[:len(groupByCols)]) } } la.possibleProperties = resultProperties diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index 7af0511c12864..143649b1f1271 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -51,7 +51,7 @@ func (a *aggregationEliminateChecker) tryToEliminateAggregation(agg *LogicalAggr return nil } } - schemaByGroupby := expression.NewSchema(agg.groupByCols...) + schemaByGroupby := expression.NewSchema(agg.GetGroupByCols()...) coveredByUniqueKey := false for _, key := range agg.children[0].Schema().Keys { if schemaByGroupby.ColumnsIndices(key) != nil { diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 0c796e4b0c3f4..0eba09a777fca 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -269,7 +269,6 @@ func (a *aggregationPushDownSolver) checkAnyCountAndSum(aggFuncs []*aggregation. func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, aggHints aggHintInfo, blockOffset int) (*LogicalAggregation, error) { agg := LogicalAggregation{ GroupByItems: expression.Column2Exprs(gbyCols), - groupByCols: gbyCols, aggHints: aggHints, }.Init(ctx, blockOffset) aggLen := len(aggFuncs) + len(gbyCols) @@ -306,7 +305,6 @@ func (a *aggregationPushDownSolver) splitPartialAgg(agg *LogicalAggregation) (pu agg.SetSchema(final.Schema) agg.AggFuncs = final.AggFuncs agg.GroupByItems = final.GroupByItems - agg.collectGroupByColumns() pushedAgg = LogicalAggregation{ AggFuncs: partial.AggFuncs, @@ -314,7 +312,6 @@ func (a *aggregationPushDownSolver) splitPartialAgg(agg *LogicalAggregation) (pu aggHints: agg.aggHints, }.Init(agg.ctx, agg.blockOffset) pushedAgg.SetSchema(partial.Schema) - pushedAgg.collectGroupByColumns() return } @@ -347,8 +344,7 @@ func (a *aggregationPushDownSolver) pushAggCrossUnion(agg *LogicalAggregation, u } newAgg.AggFuncs = append(newAgg.AggFuncs, firstRow) } - newAgg.collectGroupByColumns() - tmpSchema := expression.NewSchema(newAgg.groupByCols...) + tmpSchema := expression.NewSchema(newAgg.GetGroupByCols()...) // e.g. Union distinct will add a aggregation like `select join_agg_0, join_agg_1, join_agg_2 from t group by a, b, c` above UnionAll. // And the pushed agg will be something like `select a, b, c, a, b, c from t group by a, b, c`. So if we just return child as join does, // this will cause error during executor phase. @@ -434,7 +430,6 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan) (_ LogicalPlan, e for i, gbyItem := range agg.GroupByItems { agg.GroupByItems[i] = expression.ColumnSubstitute(gbyItem, proj.schema, proj.Exprs) } - agg.collectGroupByColumns() for _, aggFunc := range agg.AggFuncs { newArgs := make([]expression.Expression, 0, len(aggFunc.Args)) for _, arg := range aggFunc.Args { diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index 11c4e41aa24fa..4e32e518367a2 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -58,8 +58,9 @@ func (la *LogicalAggregation) BuildKeyInfo(selfSchema *expression.Schema, childS } selfSchema.Keys = append(selfSchema.Keys, newKey) } - if len(la.groupByCols) == len(la.GroupByItems) && len(la.GroupByItems) > 0 { - indices := selfSchema.ColumnsIndices(la.groupByCols) + groupByCols := la.GetGroupByCols() + if len(groupByCols) == len(la.GroupByItems) && len(la.GroupByItems) > 0 { + indices := selfSchema.ColumnsIndices(groupByCols) if indices != nil { newKey := make([]*expression.Column, 0, len(indices)) for _, i := range indices { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 12125040939f0..d482e41724e0b 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -137,9 +137,15 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) func pruneByItems(old []*util.ByItems) (new []*util.ByItems, parentUsedCols []*expression.Column) { new = make([]*util.ByItems, 0, len(old)) + seen := make(map[string]struct{}, len(old)) for _, byItem := range old { + hash := string(byItem.Expr.HashCode(nil)) + _, hashMatch := seen[hash] + seen[hash] = struct{}{} cols := expression.ExtractColumns(byItem.Expr) - if len(cols) == 0 { + if hashMatch { + // do nothing, should be filtered + } else if len(cols) == 0 { if !expression.IsRuntimeConstExpr(byItem.Expr) { new = append(new, byItem) } diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 956d651f660ff..ad28f66be5053 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -205,7 +205,6 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan) (Logica agg.SetChildren(np) // TODO: Add a Projection if any argument of aggregate funcs or group by items are scalar functions. // agg.buildProjectionIfNecessary() - agg.collectGroupByColumns() return agg, nil } // We can pull up the equal conditions below the aggregation as the join key of the apply, if only @@ -228,7 +227,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan) (Logica sel.Conditions = remainedExpr apply.CorCols = extractCorColumnsBySchema4LogicalPlan(apply.children[1], apply.children[0].Schema()) // There's no other correlated column. - groupByCols := expression.NewSchema(agg.groupByCols...) + groupByCols := expression.NewSchema(agg.GetGroupByCols()...) if len(apply.CorCols) == 0 { join := &apply.LogicalJoin join.EqualConditions = append(join.EqualConditions, eqCondWithCorCol...) @@ -250,7 +249,6 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan) (Logica groupByCols.Append(clonedCol) } } - agg.collectGroupByColumns() // The selection may be useless, check and remove it. if len(sel.Conditions) == 0 { agg.SetChildren(sel.children[0]) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 404c5e01dd77c..19326a7cb3ed4 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -206,7 +206,6 @@ func (la *LogicalAggregation) replaceExprColumns(replace map[string]*expression. for _, gbyItem := range la.GroupByItems { ResolveExprAndReplace(gbyItem, replace) } - la.collectGroupByColumns() } func (p *LogicalSelection) replaceExprColumns(replace map[string]*expression.Column) { diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 6b329358290f9..26bbc9f87a3cd 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -946,7 +946,9 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part // id as FromID. So we set the id of the newDataSource with the original one to // avoid traversing the whole plan tree to update the references. newDataSource.id = ds.id - newDataSource.statisticTable = getStatsTable(ds.SCtx(), ds.table.Meta(), pi.Definitions[i].ID) + if !ds.ctx.GetSessionVars().UseDynamicPartitionPrune() { + newDataSource.statisticTable = getStatsTable(ds.SCtx(), ds.table.Meta(), pi.Definitions[i].ID) + } err := s.resolveOptimizeHint(&newDataSource, pi.Definitions[i].Name) partitionNameSet.Insert(pi.Definitions[i].Name.L) if err != nil { diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 417375bf31a4b..e3d0195c1b1a7 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -395,7 +395,7 @@ func (la *LogicalAggregation) PredicatePushDown(predicates []expression.Expressi for _, fun := range la.AggFuncs { exprsOriginal = append(exprsOriginal, fun.Args[0]) } - groupByColumns := expression.NewSchema(la.groupByCols...) + groupByColumns := expression.NewSchema(la.GetGroupByCols()...) for _, cond := range predicates { switch cond.(type) { case *expression.Constant: diff --git a/planner/core/testdata/analyzeSuiteTestLimitIndexEstimationT.json b/planner/core/testdata/analyzeSuiteTestLimitIndexEstimationT.json new file mode 100644 index 0000000000000..1ea31af5f7629 --- /dev/null +++ b/planner/core/testdata/analyzeSuiteTestLimitIndexEstimationT.json @@ -0,0 +1,46522 @@ +{ + "database_name": "test", + "table_name": "t", + "columns": { + "a": { + "histogram": { + "ndv": 1000000, + "buckets": [ + { + "count": 3998, + "lower_bound": "MTQwMw==", + "upper_bound": "MzY0MQ==", + "repeats": 1 + }, + { + "count": 7996, + "lower_bound": "MzgxNg==", + "upper_bound": "NjUyOQ==", + "repeats": 1 + }, + { + "count": 11994, + "lower_bound": "NjgyMw==", + "upper_bound": "MTE5MTk=", + "repeats": 1 + }, + { + "count": 15992, + "lower_bound": "MTIzNTE=", + "upper_bound": "MTc4NjI=", + "repeats": 1 + }, + { + "count": 19990, + "lower_bound": "MTgxNTE=", + "upper_bound": "MjUxNjI=", + "repeats": 1 + }, + { + "count": 23988, + "lower_bound": "MjY3NDU=", + "upper_bound": "Mjk2MzE=", + "repeats": 1 + }, + { + "count": 27986, + "lower_bound": "MzAzNzA=", + "upper_bound": "MzI3Njc=", + "repeats": 1 + }, + { + "count": 31984, + "lower_bound": "MzM0NjY=", + "upper_bound": "Mzg0MjE=", + "repeats": 1 + }, + { + "count": 35982, + "lower_bound": "Mzk0NzI=", + "upper_bound": "NDI0MzQ=", + "repeats": 1 + }, + { + "count": 39980, + "lower_bound": "NDI0Njk=", + "upper_bound": "NDQ3NDI=", + "repeats": 1 + }, + { + "count": 43978, + "lower_bound": "NDgyODA=", + "upper_bound": "NTIyMjk=", + "repeats": 1 + }, + { + "count": 47976, + "lower_bound": "NTIzNTM=", + "upper_bound": "NTcwNjQ=", + "repeats": 1 + }, + { + "count": 51974, + "lower_bound": "NTcxMzc=", + "upper_bound": "NjA5ODQ=", + "repeats": 1 + }, + { + "count": 55972, + "lower_bound": "NjEyODI=", + "upper_bound": "NjU1MTk=", + "repeats": 1 + }, + { + "count": 59970, + "lower_bound": "NjU2MzA=", + "upper_bound": "Njg2ODI=", + "repeats": 1 + }, + { + "count": 63968, + "lower_bound": "NjkyMDg=", + "upper_bound": "NzczMDA=", + "repeats": 1 + }, + { + "count": 67966, + "lower_bound": "NzgyMTc=", + "upper_bound": "ODMyODc=", + "repeats": 1 + }, + { + "count": 71964, + "lower_bound": "ODM4NjM=", + "upper_bound": "ODgyODQ=", + "repeats": 1 + }, + { + "count": 75962, + "lower_bound": "ODg3NDc=", + "upper_bound": "OTE1MTQ=", + "repeats": 1 + }, + { + "count": 79960, + "lower_bound": "OTE5NDk=", + "upper_bound": "OTc0MjY=", + "repeats": 1 + }, + { + "count": 83958, + "lower_bound": "OTc1NTA=", + "upper_bound": "MTAyMjU2", + "repeats": 1 + }, + { + "count": 87956, + "lower_bound": "MTAzODAy", + "upper_bound": "MTA4ODM5", + "repeats": 1 + }, + { + "count": 91954, + "lower_bound": "MTA5NDA0", + "upper_bound": "MTExODEy", + "repeats": 1 + }, + { + "count": 95952, + "lower_bound": "MTEyMjEw", + "upper_bound": "MTE4NTEx", + "repeats": 1 + }, + { + "count": 99950, + "lower_bound": "MTE5MDE2", + "upper_bound": "MTI0MjQw", + "repeats": 1 + }, + { + "count": 103948, + "lower_bound": "MTI0ODk1", + "upper_bound": "MTMwNDM0", + "repeats": 1 + }, + { + "count": 107946, + "lower_bound": "MTMwODU1", + "upper_bound": "MTMzODI1", + "repeats": 1 + }, + { + "count": 111944, + "lower_bound": "MTM1NDkz", + "upper_bound": "MTM3MzA3", + "repeats": 1 + }, + { + "count": 115942, + "lower_bound": "MTM4MjQ3", + "upper_bound": "MTM5Njk3", + "repeats": 1 + }, + { + "count": 119940, + "lower_bound": "MTQwMzQ2", + "upper_bound": "MTQyMTg4", + "repeats": 1 + }, + { + "count": 123938, + "lower_bound": "MTQyNzgy", + "upper_bound": "MTQ1NDIz", + "repeats": 1 + }, + { + "count": 127936, + "lower_bound": "MTQ2NjQ4", + "upper_bound": "MTUxMDk5", + "repeats": 1 + }, + { + "count": 131934, + "lower_bound": "MTUxNTM1", + "upper_bound": "MTU2MzI5", + "repeats": 1 + }, + { + "count": 135932, + "lower_bound": "MTU2NDI0", + "upper_bound": "MTY0MTMx", + "repeats": 1 + }, + { + "count": 139930, + "lower_bound": "MTY1NDE2", + "upper_bound": "MTY3NTY3", + "repeats": 1 + }, + { + "count": 143928, + "lower_bound": "MTY4Mzcw", + "upper_bound": "MTcyNzQz", + "repeats": 1 + }, + { + "count": 147926, + "lower_bound": "MTczMDU3", + "upper_bound": "MTc2MzI3", + "repeats": 1 + }, + { + "count": 151924, + "lower_bound": "MTc2MzYz", + "upper_bound": "MTgzMDIz", + "repeats": 1 + }, + { + "count": 155922, + "lower_bound": "MTgzNTI1", + "upper_bound": "MTkwMTg2", + "repeats": 1 + }, + { + "count": 159920, + "lower_bound": "MTkwNzQx", + "upper_bound": "MTk1MDcz", + "repeats": 1 + }, + { + "count": 163918, + "lower_bound": "MTk1MjQ5", + "upper_bound": "MTk4Mzk4", + "repeats": 1 + }, + { + "count": 167916, + "lower_bound": "MTk4NjQx", + "upper_bound": "MjA2OTYz", + "repeats": 1 + }, + { + "count": 171914, + "lower_bound": "MjA3MTM5", + "upper_bound": "MjEwNzg0", + "repeats": 1 + }, + { + "count": 175912, + "lower_bound": "MjExMzky", + "upper_bound": "MjE3OTk5", + "repeats": 1 + }, + { + "count": 179910, + "lower_bound": "MjE4MTU2", + "upper_bound": "MjI2Mzkw", + "repeats": 1 + }, + { + "count": 183908, + "lower_bound": "MjI3NDU2", + "upper_bound": "MjMwNDI1", + "repeats": 1 + }, + { + "count": 187906, + "lower_bound": "MjMwNDM4", + "upper_bound": "MjM0NjMy", + "repeats": 1 + }, + { + "count": 191904, + "lower_bound": "MjM1NjIw", + "upper_bound": "MjQxMDk3", + "repeats": 1 + }, + { + "count": 195902, + "lower_bound": "MjQxMTAy", + "upper_bound": "MjQ0MjYy", + "repeats": 1 + }, + { + "count": 199900, + "lower_bound": "MjQ1Njgx", + "upper_bound": "MjUzNTY1", + "repeats": 1 + }, + { + "count": 203898, + "lower_bound": "MjU0NTg4", + "upper_bound": "MjU5NzQx", + "repeats": 1 + }, + { + "count": 207896, + "lower_bound": "MjYwNTM3", + "upper_bound": "MjYzNzY4", + "repeats": 1 + }, + { + "count": 211894, + "lower_bound": "MjY0MDU5", + "upper_bound": "MjY5Mzk0", + "repeats": 1 + }, + { + "count": 215892, + "lower_bound": "MjY5NDU4", + "upper_bound": "MjczODc1", + "repeats": 1 + }, + { + "count": 219890, + "lower_bound": "MjczODk4", + "upper_bound": "MjgwNDk4", + "repeats": 1 + }, + { + "count": 223888, + "lower_bound": "MjgwNTg3", + "upper_bound": "Mjg2MDE3", + "repeats": 1 + }, + { + "count": 227886, + "lower_bound": "Mjg4ODI1", + "upper_bound": "MjkzNzIw", + "repeats": 1 + }, + { + "count": 231884, + "lower_bound": "Mjk0NjAz", + "upper_bound": "Mjk3Nzg3", + "repeats": 1 + }, + { + "count": 235882, + "lower_bound": "Mjk4Mjgz", + "upper_bound": "MzAzMzQ2", + "repeats": 1 + }, + { + "count": 239880, + "lower_bound": "MzA0OTcy", + "upper_bound": "MzA4OTgy", + "repeats": 1 + }, + { + "count": 243878, + "lower_bound": "MzA5NDky", + "upper_bound": "MzE1MjA4", + "repeats": 1 + }, + { + "count": 247876, + "lower_bound": "MzE2MjQx", + "upper_bound": "MzE5MDYz", + "repeats": 1 + }, + { + "count": 251874, + "lower_bound": "MzE5MTgy", + "upper_bound": "MzIxODcz", + "repeats": 1 + }, + { + "count": 255872, + "lower_bound": "MzIyODQ5", + "upper_bound": "MzI1NjQw", + "repeats": 1 + }, + { + "count": 259870, + "lower_bound": "MzI1OTQ2", + "upper_bound": "MzMxMTY3", + "repeats": 1 + }, + { + "count": 263868, + "lower_bound": "MzMxODU1", + "upper_bound": "MzM1NTMw", + "repeats": 1 + }, + { + "count": 267866, + "lower_bound": "MzM2Mjg1", + "upper_bound": "MzQyMzE3", + "repeats": 1 + }, + { + "count": 271864, + "lower_bound": "MzQyNjAw", + "upper_bound": "MzQ3MjU2", + "repeats": 1 + }, + { + "count": 275862, + "lower_bound": "MzQ4MTIw", + "upper_bound": "MzUxMjg4", + "repeats": 1 + }, + { + "count": 279860, + "lower_bound": "MzUzODc1", + "upper_bound": "MzU3NjI2", + "repeats": 1 + }, + { + "count": 283858, + "lower_bound": "MzU3NzAx", + "upper_bound": "MzYyMjIy", + "repeats": 1 + }, + { + "count": 287856, + "lower_bound": "MzYzNDgy", + "upper_bound": "MzcwNzQx", + "repeats": 1 + }, + { + "count": 291854, + "lower_bound": "MzcxMDk3", + "upper_bound": "Mzc0ODky", + "repeats": 1 + }, + { + "count": 295852, + "lower_bound": "Mzc1NDkw", + "upper_bound": "MzgwMzU3", + "repeats": 1 + }, + { + "count": 299850, + "lower_bound": "MzgxNDQ1", + "upper_bound": "Mzg0ODA0", + "repeats": 1 + }, + { + "count": 303848, + "lower_bound": "Mzg0ODYw", + "upper_bound": "Mzg5NDQz", + "repeats": 1 + }, + { + "count": 307846, + "lower_bound": "MzkwNDUw", + "upper_bound": "Mzk1MTU5", + "repeats": 1 + }, + { + "count": 311844, + "lower_bound": "Mzk2Nzgz", + "upper_bound": "NDAwNDUy", + "repeats": 1 + }, + { + "count": 315842, + "lower_bound": "NDAwNTIz", + "upper_bound": "NDAzNDc1", + "repeats": 1 + }, + { + "count": 319840, + "lower_bound": "NDAzOTQz", + "upper_bound": "NDEwNDg0", + "repeats": 1 + }, + { + "count": 323838, + "lower_bound": "NDEwODY0", + "upper_bound": "NDE1NDUy", + "repeats": 1 + }, + { + "count": 327836, + "lower_bound": "NDE2MTQ4", + "upper_bound": "NDE4MDkx", + "repeats": 1 + }, + { + "count": 331834, + "lower_bound": "NDE4MTM3", + "upper_bound": "NDI0NzIz", + "repeats": 1 + }, + { + "count": 335832, + "lower_bound": "NDI0NzI0", + "upper_bound": "NDI4OTAy", + "repeats": 1 + }, + { + "count": 339830, + "lower_bound": "NDI5MzE3", + "upper_bound": "NDMzNjI3", + "repeats": 1 + }, + { + "count": 343828, + "lower_bound": "NDMzODM4", + "upper_bound": "NDM3MjMz", + "repeats": 1 + }, + { + "count": 347826, + "lower_bound": "NDM4MDIz", + "upper_bound": "NDQyMzcz", + "repeats": 1 + }, + { + "count": 351824, + "lower_bound": "NDQzMzA1", + "upper_bound": "NDQ3Mzcx", + "repeats": 1 + }, + { + "count": 355822, + "lower_bound": "NDQ4MzM3", + "upper_bound": "NDUyODk0", + "repeats": 1 + }, + { + "count": 359820, + "lower_bound": "NDUzMzY4", + "upper_bound": "NDU2MTI4", + "repeats": 1 + }, + { + "count": 363818, + "lower_bound": "NDU2MzU5", + "upper_bound": "NDYwNzkz", + "repeats": 1 + }, + { + "count": 367816, + "lower_bound": "NDYxMzIx", + "upper_bound": "NDcxOTM4", + "repeats": 1 + }, + { + "count": 371814, + "lower_bound": "NDcyMjMz", + "upper_bound": "NDc5MDA3", + "repeats": 1 + }, + { + "count": 375812, + "lower_bound": "NDc5NDg1", + "upper_bound": "NDgyMTIx", + "repeats": 1 + }, + { + "count": 379810, + "lower_bound": "NDgzMjM1", + "upper_bound": "NDg2MTY1", + "repeats": 1 + }, + { + "count": 383808, + "lower_bound": "NDg4MjM2", + "upper_bound": "NDkxMTky", + "repeats": 1 + }, + { + "count": 387806, + "lower_bound": "NDkyNDA0", + "upper_bound": "NDk2OTA1", + "repeats": 1 + }, + { + "count": 391804, + "lower_bound": "NDk3ODY2", + "upper_bound": "NTAyNzk4", + "repeats": 1 + }, + { + "count": 395802, + "lower_bound": "NTAzODI1", + "upper_bound": "NTA2NTc0", + "repeats": 1 + }, + { + "count": 399800, + "lower_bound": "NTA2OTQw", + "upper_bound": "NTA5ODA1", + "repeats": 1 + }, + { + "count": 403798, + "lower_bound": "NTA5ODQx", + "upper_bound": "NTE2OTU3", + "repeats": 1 + }, + { + "count": 407796, + "lower_bound": "NTE3NTM3", + "upper_bound": "NTIwMTIz", + "repeats": 1 + }, + { + "count": 411794, + "lower_bound": "NTIwMjQ5", + "upper_bound": "NTI0Mzk4", + "repeats": 1 + }, + { + "count": 415792, + "lower_bound": "NTI2MTEy", + "upper_bound": "NTMyOTU0", + "repeats": 1 + }, + { + "count": 419790, + "lower_bound": "NTMzNjk1", + "upper_bound": "NTM2ODQ4", + "repeats": 1 + }, + { + "count": 423788, + "lower_bound": "NTM2OTEz", + "upper_bound": "NTQyMzU1", + "repeats": 1 + }, + { + "count": 427786, + "lower_bound": "NTQyNTQ2", + "upper_bound": "NTQ2ODE3", + "repeats": 1 + }, + { + "count": 431784, + "lower_bound": "NTQ4OTA3", + "upper_bound": "NTUyNDQw", + "repeats": 1 + }, + { + "count": 435782, + "lower_bound": "NTUyODI2", + "upper_bound": "NTU2MDE2", + "repeats": 1 + }, + { + "count": 439780, + "lower_bound": "NTU2MjMx", + "upper_bound": "NTYwNzU3", + "repeats": 1 + }, + { + "count": 443778, + "lower_bound": "NTYxMjI0", + "upper_bound": "NTY3ODg1", + "repeats": 1 + }, + { + "count": 447776, + "lower_bound": "NTY4MDAx", + "upper_bound": "NTczMjY4", + "repeats": 1 + }, + { + "count": 451774, + "lower_bound": "NTczMzYx", + "upper_bound": "NTc5MTA4", + "repeats": 1 + }, + { + "count": 455772, + "lower_bound": "NTc5MzQx", + "upper_bound": "NTg1ODk3", + "repeats": 1 + }, + { + "count": 459770, + "lower_bound": "NTg2ODE4", + "upper_bound": "NTkwMjQz", + "repeats": 1 + }, + { + "count": 463768, + "lower_bound": "NTkwNDYz", + "upper_bound": "NTkyNzEz", + "repeats": 1 + }, + { + "count": 467766, + "lower_bound": "NTkzNTEz", + "upper_bound": "NTk1NDgz", + "repeats": 1 + }, + { + "count": 471764, + "lower_bound": "NTk3MDEz", + "upper_bound": "NjAxMTU5", + "repeats": 1 + }, + { + "count": 475762, + "lower_bound": "NjAxMTk5", + "upper_bound": "NjA2MjI1", + "repeats": 1 + }, + { + "count": 479760, + "lower_bound": "NjA3NzU4", + "upper_bound": "NjEwNTMy", + "repeats": 1 + }, + { + "count": 483758, + "lower_bound": "NjEyOTI2", + "upper_bound": "NjE3ODcz", + "repeats": 1 + }, + { + "count": 487756, + "lower_bound": "NjE3OTQz", + "upper_bound": "NjIyMjI1", + "repeats": 1 + }, + { + "count": 491754, + "lower_bound": "NjIyNTg3", + "upper_bound": "NjI2NTI1", + "repeats": 1 + }, + { + "count": 495752, + "lower_bound": "NjI2NTQ2", + "upper_bound": "NjI5ODk4", + "repeats": 1 + }, + { + "count": 499750, + "lower_bound": "NjI5OTE0", + "upper_bound": "NjMzNzcy", + "repeats": 1 + }, + { + "count": 503748, + "lower_bound": "NjM0MDc3", + "upper_bound": "NjM2ODI4", + "repeats": 1 + }, + { + "count": 507746, + "lower_bound": "NjM2OTA2", + "upper_bound": "NjM5NDg3", + "repeats": 1 + }, + { + "count": 511744, + "lower_bound": "NjM5NjYz", + "upper_bound": "NjQwODE2", + "repeats": 1 + }, + { + "count": 515742, + "lower_bound": "NjQxMjY5", + "upper_bound": "NjQzNzcw", + "repeats": 1 + }, + { + "count": 519740, + "lower_bound": "NjQzODY0", + "upper_bound": "NjQ1NTEw", + "repeats": 1 + }, + { + "count": 523738, + "lower_bound": "NjQ2MjY0", + "upper_bound": "NjUwMzQ1", + "repeats": 1 + }, + { + "count": 527736, + "lower_bound": "NjUwNjY1", + "upper_bound": "NjUxODYx", + "repeats": 1 + }, + { + "count": 531734, + "lower_bound": "NjUyMDcy", + "upper_bound": "NjU2Mjgz", + "repeats": 1 + }, + { + "count": 535732, + "lower_bound": "NjU2NTYz", + "upper_bound": "NjU3ODk2", + "repeats": 1 + }, + { + "count": 539730, + "lower_bound": "NjU4NzQ3", + "upper_bound": "NjYxOTYy", + "repeats": 1 + }, + { + "count": 543728, + "lower_bound": "NjYzNDEw", + "upper_bound": "NjY2MDE3", + "repeats": 1 + }, + { + "count": 547726, + "lower_bound": "NjY2Mzg0", + "upper_bound": "NjY5NTkz", + "repeats": 1 + }, + { + "count": 551724, + "lower_bound": "NjY5NzI2", + "upper_bound": "NjcyOTEx", + "repeats": 1 + }, + { + "count": 555722, + "lower_bound": "NjczMDY1", + "upper_bound": "Njc1Njg0", + "repeats": 1 + }, + { + "count": 559720, + "lower_bound": "Njc1NzYx", + "upper_bound": "Njc4NDM1", + "repeats": 1 + }, + { + "count": 563718, + "lower_bound": "Njc4NTc4", + "upper_bound": "Njc5NzQ1", + "repeats": 1 + }, + { + "count": 567716, + "lower_bound": "NjgwMjc4", + "upper_bound": "NjgzMTA5", + "repeats": 1 + }, + { + "count": 571714, + "lower_bound": "NjgzMTEx", + "upper_bound": "Njg1MjA5", + "repeats": 1 + }, + { + "count": 575712, + "lower_bound": "Njg1Mzg2", + "upper_bound": "Njg2ODgy", + "repeats": 1 + }, + { + "count": 579710, + "lower_bound": "Njg2OTE3", + "upper_bound": "Njg5MjE5", + "repeats": 1 + }, + { + "count": 583708, + "lower_bound": "Njg5MjQ0", + "upper_bound": "NjkxMjkw", + "repeats": 1 + }, + { + "count": 587706, + "lower_bound": "NjkxNDgy", + "upper_bound": "Njk1Mzk0", + "repeats": 1 + }, + { + "count": 591704, + "lower_bound": "Njk2MTMw", + "upper_bound": "Njk4MDMw", + "repeats": 1 + }, + { + "count": 595702, + "lower_bound": "Njk4OTMx", + "upper_bound": "NzAyMzE4", + "repeats": 1 + }, + { + "count": 599700, + "lower_bound": "NzAzNzcx", + "upper_bound": "NzA3MjUw", + "repeats": 1 + }, + { + "count": 603698, + "lower_bound": "NzA3Mzkx", + "upper_bound": "NzA4MzA4", + "repeats": 1 + }, + { + "count": 607696, + "lower_bound": "NzA4NTUx", + "upper_bound": "NzExNjE1", + "repeats": 1 + }, + { + "count": 611694, + "lower_bound": "NzExNjM0", + "upper_bound": "NzE0OTMy", + "repeats": 1 + }, + { + "count": 615692, + "lower_bound": "NzE1MTAy", + "upper_bound": "NzE2MzUx", + "repeats": 1 + }, + { + "count": 619690, + "lower_bound": "NzE2NTkw", + "upper_bound": "NzE5MTA5", + "repeats": 1 + }, + { + "count": 623688, + "lower_bound": "NzE5NTQ2", + "upper_bound": "NzIyNzIw", + "repeats": 1 + }, + { + "count": 627686, + "lower_bound": "NzIyNzY3", + "upper_bound": "NzI4Mzk4", + "repeats": 1 + }, + { + "count": 631684, + "lower_bound": "NzI4NDc0", + "upper_bound": "NzMwODE1", + "repeats": 1 + }, + { + "count": 635682, + "lower_bound": "NzMwODIz", + "upper_bound": "NzMyMDg4", + "repeats": 1 + }, + { + "count": 639680, + "lower_bound": "NzMyOTI4", + "upper_bound": "NzM1OTM5", + "repeats": 1 + }, + { + "count": 643678, + "lower_bound": "NzM2MDIx", + "upper_bound": "NzM4NzI4", + "repeats": 1 + }, + { + "count": 647676, + "lower_bound": "NzM5NjU5", + "upper_bound": "NzQxMzc1", + "repeats": 1 + }, + { + "count": 651674, + "lower_bound": "NzQyNzcw", + "upper_bound": "NzQ0NDky", + "repeats": 1 + }, + { + "count": 655672, + "lower_bound": "NzQ1NDI2", + "upper_bound": "NzQ5MDc0", + "repeats": 1 + }, + { + "count": 659670, + "lower_bound": "NzQ5MTE4", + "upper_bound": "NzUxMDA5", + "repeats": 1 + }, + { + "count": 663668, + "lower_bound": "NzUxMzQ4", + "upper_bound": "NzU0Nzg2", + "repeats": 1 + }, + { + "count": 667666, + "lower_bound": "NzU0ODEz", + "upper_bound": "NzU3MDA3", + "repeats": 1 + }, + { + "count": 671664, + "lower_bound": "NzU3MDUz", + "upper_bound": "NzYwNjIy", + "repeats": 1 + }, + { + "count": 675662, + "lower_bound": "NzYwNjc3", + "upper_bound": "NzYzOTk3", + "repeats": 1 + }, + { + "count": 679660, + "lower_bound": "NzY0NTEx", + "upper_bound": "NzY2ODcw", + "repeats": 1 + }, + { + "count": 683658, + "lower_bound": "NzY3Mzcy", + "upper_bound": "NzcwOTU1", + "repeats": 1 + }, + { + "count": 687656, + "lower_bound": "NzcxMDU2", + "upper_bound": "NzczNDk2", + "repeats": 1 + }, + { + "count": 691654, + "lower_bound": "NzczNjU3", + "upper_bound": "Nzc2OTMz", + "repeats": 1 + }, + { + "count": 695652, + "lower_bound": "Nzc3MTE0", + "upper_bound": "Nzc5NDU0", + "repeats": 1 + }, + { + "count": 699650, + "lower_bound": "Nzc5NTA3", + "upper_bound": "NzgxMDgw", + "repeats": 1 + }, + { + "count": 703648, + "lower_bound": "NzgxMTkx", + "upper_bound": "NzgyOTIy", + "repeats": 1 + }, + { + "count": 707646, + "lower_bound": "NzgzMTMx", + "upper_bound": "Nzg1NTg2", + "repeats": 1 + }, + { + "count": 711644, + "lower_bound": "Nzg2MTY4", + "upper_bound": "NzkxODI5", + "repeats": 1 + }, + { + "count": 715642, + "lower_bound": "NzkxODgw", + "upper_bound": "Nzk0NDE3", + "repeats": 1 + }, + { + "count": 719640, + "lower_bound": "Nzk0Nzc2", + "upper_bound": "Nzk5MDE0", + "repeats": 1 + }, + { + "count": 723638, + "lower_bound": "Nzk5MDYw", + "upper_bound": "ODAxOTgw", + "repeats": 1 + }, + { + "count": 727636, + "lower_bound": "ODAyMDM5", + "upper_bound": "ODA0MjQ2", + "repeats": 1 + }, + { + "count": 731634, + "lower_bound": "ODA0NjE5", + "upper_bound": "ODA2OTMx", + "repeats": 1 + }, + { + "count": 735632, + "lower_bound": "ODA3MzMw", + "upper_bound": "ODA4NDc3", + "repeats": 1 + }, + { + "count": 739630, + "lower_bound": "ODA5MDU5", + "upper_bound": "ODEwOTY2", + "repeats": 1 + }, + { + "count": 743628, + "lower_bound": "ODEwOTcz", + "upper_bound": "ODEyMTEy", + "repeats": 1 + }, + { + "count": 747626, + "lower_bound": "ODEyNzcy", + "upper_bound": "ODE2ODI1", + "repeats": 1 + }, + { + "count": 751624, + "lower_bound": "ODE3MTY2", + "upper_bound": "ODIwMTU2", + "repeats": 1 + }, + { + "count": 755622, + "lower_bound": "ODIwNDUx", + "upper_bound": "ODIzMjc5", + "repeats": 1 + }, + { + "count": 759620, + "lower_bound": "ODIzNDM4", + "upper_bound": "ODI2MjEz", + "repeats": 1 + }, + { + "count": 763618, + "lower_bound": "ODI2NTMx", + "upper_bound": "ODI4MTgz", + "repeats": 1 + }, + { + "count": 767616, + "lower_bound": "ODI4MjUw", + "upper_bound": "ODI5MjQ5", + "repeats": 1 + }, + { + "count": 771614, + "lower_bound": "ODI5NDYy", + "upper_bound": "ODMxODU1", + "repeats": 1 + }, + { + "count": 775612, + "lower_bound": "ODMyMjc1", + "upper_bound": "ODM1NTI0", + "repeats": 1 + }, + { + "count": 779610, + "lower_bound": "ODM1Nzgz", + "upper_bound": "ODM3OTcx", + "repeats": 1 + }, + { + "count": 783608, + "lower_bound": "ODM4ODM4", + "upper_bound": "ODQwMjUw", + "repeats": 1 + }, + { + "count": 787606, + "lower_bound": "ODQwMzA3", + "upper_bound": "ODQyMTg0", + "repeats": 1 + }, + { + "count": 791604, + "lower_bound": "ODQyMjA3", + "upper_bound": "ODQ2NDcx", + "repeats": 1 + }, + { + "count": 795602, + "lower_bound": "ODQ3MDY3", + "upper_bound": "ODUyMDYz", + "repeats": 1 + }, + { + "count": 799600, + "lower_bound": "ODUyMDY1", + "upper_bound": "ODU0MDU2", + "repeats": 1 + }, + { + "count": 803598, + "lower_bound": "ODU0MTQ1", + "upper_bound": "ODU2NTY2", + "repeats": 1 + }, + { + "count": 807596, + "lower_bound": "ODU2NzMy", + "upper_bound": "ODU4MzUy", + "repeats": 1 + }, + { + "count": 811594, + "lower_bound": "ODU4NTY4", + "upper_bound": "ODYxOTg0", + "repeats": 1 + }, + { + "count": 815592, + "lower_bound": "ODYyMDEx", + "upper_bound": "ODY1Mjgx", + "repeats": 1 + }, + { + "count": 819590, + "lower_bound": "ODY1MzE4", + "upper_bound": "ODY4MTE5", + "repeats": 1 + }, + { + "count": 823588, + "lower_bound": "ODY4MzY2", + "upper_bound": "ODcwODY3", + "repeats": 1 + }, + { + "count": 827586, + "lower_bound": "ODcwOTYx", + "upper_bound": "ODcyMjYz", + "repeats": 1 + }, + { + "count": 831584, + "lower_bound": "ODcyNDQx", + "upper_bound": "ODc0MjE0", + "repeats": 1 + }, + { + "count": 835582, + "lower_bound": "ODc0ODA2", + "upper_bound": "ODc3MzEy", + "repeats": 1 + }, + { + "count": 839580, + "lower_bound": "ODc3NTEx", + "upper_bound": "ODgwNzQ2", + "repeats": 1 + }, + { + "count": 843578, + "lower_bound": "ODgxMjI5", + "upper_bound": "ODgyNTY1", + "repeats": 1 + }, + { + "count": 847576, + "lower_bound": "ODgzNjc4", + "upper_bound": "ODg1MTc1", + "repeats": 1 + }, + { + "count": 851574, + "lower_bound": "ODg2MzMx", + "upper_bound": "ODg3ODgx", + "repeats": 1 + }, + { + "count": 855572, + "lower_bound": "ODg3OTUz", + "upper_bound": "ODkwNTI0", + "repeats": 1 + }, + { + "count": 859570, + "lower_bound": "ODkwNjk0", + "upper_bound": "ODkzMDY2", + "repeats": 1 + }, + { + "count": 863568, + "lower_bound": "ODkzMTYy", + "upper_bound": "ODk3Mzcy", + "repeats": 1 + }, + { + "count": 867566, + "lower_bound": "ODk3NjIw", + "upper_bound": "ODk5OTY1", + "repeats": 1 + }, + { + "count": 871564, + "lower_bound": "ODk5OTk4", + "upper_bound": "OTAxOTA0", + "repeats": 1 + }, + { + "count": 875562, + "lower_bound": "OTAyMDY3", + "upper_bound": "OTA1MTU5", + "repeats": 1 + }, + { + "count": 879560, + "lower_bound": "OTA1MjY4", + "upper_bound": "OTA3Mjk2", + "repeats": 1 + }, + { + "count": 883558, + "lower_bound": "OTA4MjQ4", + "upper_bound": "OTEzODgy", + "repeats": 1 + }, + { + "count": 887556, + "lower_bound": "OTE1OTI1", + "upper_bound": "OTE4NDQ4", + "repeats": 1 + }, + { + "count": 891554, + "lower_bound": "OTE4NTU4", + "upper_bound": "OTIwMTc3", + "repeats": 1 + }, + { + "count": 895552, + "lower_bound": "OTIwNDgy", + "upper_bound": "OTIzNjY5", + "repeats": 1 + }, + { + "count": 899550, + "lower_bound": "OTIzNzkx", + "upper_bound": "OTI2NDg0", + "repeats": 1 + }, + { + "count": 903548, + "lower_bound": "OTI2NjA4", + "upper_bound": "OTI3NzQz", + "repeats": 1 + }, + { + "count": 907546, + "lower_bound": "OTI4NDY5", + "upper_bound": "OTMwMzMz", + "repeats": 1 + }, + { + "count": 911544, + "lower_bound": "OTMwNzEy", + "upper_bound": "OTMyNzE0", + "repeats": 1 + }, + { + "count": 915542, + "lower_bound": "OTMyOTI4", + "upper_bound": "OTM0MTU1", + "repeats": 1 + }, + { + "count": 919540, + "lower_bound": "OTM0Njc4", + "upper_bound": "OTM2NTM5", + "repeats": 1 + }, + { + "count": 923538, + "lower_bound": "OTM2OTMz", + "upper_bound": "OTM4NTU4", + "repeats": 1 + }, + { + "count": 927536, + "lower_bound": "OTM5NDc2", + "upper_bound": "OTQyNjY2", + "repeats": 1 + }, + { + "count": 931534, + "lower_bound": "OTQyNzkz", + "upper_bound": "OTQ1MDY5", + "repeats": 1 + }, + { + "count": 935532, + "lower_bound": "OTQ1OTMz", + "upper_bound": "OTQ4OTY5", + "repeats": 1 + }, + { + "count": 939530, + "lower_bound": "OTUwMDEz", + "upper_bound": "OTUzMTAz", + "repeats": 1 + }, + { + "count": 943528, + "lower_bound": "OTUzMTIx", + "upper_bound": "OTU1NzY3", + "repeats": 1 + }, + { + "count": 947526, + "lower_bound": "OTU2MzMy", + "upper_bound": "OTU3NTQ1", + "repeats": 1 + }, + { + "count": 951524, + "lower_bound": "OTU4MDk3", + "upper_bound": "OTY0OTA3", + "repeats": 1 + }, + { + "count": 955522, + "lower_bound": "OTY1NDQ0", + "upper_bound": "OTY3NDg4", + "repeats": 1 + }, + { + "count": 959520, + "lower_bound": "OTY3NjQ1", + "upper_bound": "OTcxNDI0", + "repeats": 1 + }, + { + "count": 963518, + "lower_bound": "OTcxNTEx", + "upper_bound": "OTczNDI5", + "repeats": 1 + }, + { + "count": 967516, + "lower_bound": "OTczNDk0", + "upper_bound": "OTc0NTk2", + "repeats": 1 + }, + { + "count": 971514, + "lower_bound": "OTc0ODAy", + "upper_bound": "OTc2ODQ5", + "repeats": 1 + }, + { + "count": 975512, + "lower_bound": "OTc3MjAz", + "upper_bound": "OTc4MjU4", + "repeats": 1 + }, + { + "count": 979510, + "lower_bound": "OTc4NDA3", + "upper_bound": "OTgxNDQw", + "repeats": 1 + }, + { + "count": 983508, + "lower_bound": "OTgxNTA3", + "upper_bound": "OTg3MTkw", + "repeats": 1 + }, + { + "count": 987506, + "lower_bound": "OTg3NjM3", + "upper_bound": "OTg5Mzc0", + "repeats": 1 + }, + { + "count": 991504, + "lower_bound": "OTg5NjUw", + "upper_bound": "OTkzMjk0", + "repeats": 1 + }, + { + "count": 995502, + "lower_bound": "OTkzMzUz", + "upper_bound": "OTk0Nzc1", + "repeats": 1 + }, + { + "count": 999500, + "lower_bound": "OTk1NDYz", + "upper_bound": "OTk4MDgw", + "repeats": 1 + }, + { + "count": 1000000, + "lower_bound": "OTk5NjU4", + "upper_bound": "OTk5NjU4", + "repeats": 1 + } + ] + }, + "cm_sketch": { + "rows": [ + { + "counters": [ + 495, + 474, + 474, + 471, + 509, + 474, + 509, + 527, + 490, + 509, + 475, + 503, + 506, + 489, + 472, + 504, + 513, + 469, + 508, + 480, + 476, + 529, + 488, + 486, + 513, + 477, + 507, + 477, + 514, + 533, + 456, + 512, + 460, + 474, + 473, + 489, + 472, + 517, + 459, + 516, + 489, + 491, + 497, + 492, + 472, + 508, + 498, + 496, + 438, + 472, + 485, + 452, + 484, + 493, + 477, + 471, + 534, + 500, + 466, + 499, + 476, + 458, + 493, + 481, + 497, + 473, + 479, + 490, + 469, + 519, + 471, + 483, + 482, + 489, + 506, + 544, + 504, + 492, + 457, + 506, + 479, + 477, + 491, + 464, + 496, + 509, + 492, + 471, + 440, + 479, + 476, + 505, + 522, + 498, + 442, + 471, + 492, + 499, + 516, + 494, + 477, + 493, + 500, + 478, + 496, + 482, + 508, + 500, + 495, + 490, + 456, + 501, + 541, + 469, + 507, + 497, + 477, + 450, + 515, + 472, + 501, + 500, + 514, + 480, + 459, + 476, + 470, + 491, + 493, + 520, + 535, + 508, + 494, + 549, + 501, + 489, + 528, + 511, + 488, + 449, + 455, + 452, + 482, + 474, + 514, + 472, + 445, + 476, + 505, + 484, + 466, + 489, + 510, + 495, + 489, + 479, + 467, + 521, + 480, + 500, + 501, + 495, + 457, + 457, + 487, + 487, + 502, + 477, + 472, + 498, + 473, + 489, + 486, + 480, + 531, + 480, + 500, + 510, + 505, + 499, + 477, + 473, + 506, + 475, + 494, + 496, + 447, + 490, + 500, + 474, + 513, + 512, + 470, + 480, + 480, + 539, + 476, + 506, + 503, + 481, + 526, + 493, + 449, + 500, + 467, + 497, + 461, + 497, + 502, + 468, + 475, + 497, + 501, + 510, + 469, + 477, + 439, + 497, + 469, + 494, + 492, + 482, + 495, + 470, + 438, + 512, + 485, + 525, + 460, + 541, + 500, + 476, + 502, + 510, + 476, + 482, + 520, + 435, + 447, + 487, + 487, + 461, + 485, + 461, + 488, + 531, + 519, + 476, + 444, + 490, + 510, + 506, + 492, + 465, + 507, + 512, + 517, + 492, + 481, + 490, + 488, + 487, + 453, + 490, + 536, + 494, + 515, + 504, + 504, + 510, + 511, + 489, + 478, + 461, + 522, + 503, + 495, + 508, + 469, + 454, + 522, + 537, + 482, + 506, + 493, + 501, + 513, + 450, + 493, + 476, + 499, + 491, + 468, + 504, + 524, + 488, + 469, + 478, + 457, + 473, + 481, + 469, + 479, + 491, + 488, + 483, + 509, + 523, + 517, + 497, + 520, + 491, + 453, + 520, + 508, + 486, + 488, + 489, + 543, + 469, + 498, + 471, + 472, + 495, + 507, + 473, + 497, + 487, + 493, + 462, + 475, + 474, + 490, + 518, + 492, + 462, + 488, + 497, + 474, + 502, + 550, + 490, + 452, + 492, + 465, + 488, + 467, + 490, + 513, + 448, + 510, + 490, + 459, + 477, + 487, + 500, + 501, + 505, + 491, + 506, + 443, + 482, + 466, + 482, + 459, + 461, + 523, + 502, + 466, + 469, + 504, + 494, + 483, + 440, + 497, + 499, + 483, + 500, + 484, + 500, + 527, + 454, + 493, + 445, + 474, + 486, + 507, + 490, + 479, + 493, + 459, + 474, + 492, + 482, + 497, + 483, + 516, + 499, + 462, + 522, + 482, + 498, + 486, + 464, + 504, + 496, + 498, + 493, + 481, + 462, + 508, + 484, + 485, + 479, + 487, + 489, + 493, + 471, + 495, + 541, + 495, + 472, + 482, + 461, + 471, + 501, + 480, + 460, + 503, + 500, + 511, + 492, + 491, + 450, + 520, + 462, + 510, + 499, + 460, + 515, + 519, + 455, + 495, + 470, + 464, + 509, + 479, + 456, + 466, + 523, + 478, + 461, + 496, + 500, + 469, + 506, + 512, + 490, + 523, + 485, + 452, + 528, + 486, + 458, + 468, + 476, + 510, + 500, + 499, + 485, + 478, + 526, + 526, + 493, + 478, + 491, + 503, + 467, + 506, + 478, + 501, + 453, + 520, + 499, + 469, + 479, + 479, + 497, + 483, + 492, + 476, + 511, + 469, + 510, + 508, + 457, + 482, + 475, + 504, + 462, + 508, + 455, + 518, + 516, + 485, + 505, + 478, + 504, + 476, + 486, + 484, + 495, + 510, + 476, + 491, + 469, + 450, + 472, + 497, + 470, + 491, + 504, + 469, + 479, + 481, + 521, + 470, + 524, + 508, + 487, + 466, + 427, + 544, + 489, + 491, + 471, + 505, + 497, + 469, + 478, + 490, + 461, + 462, + 446, + 487, + 492, + 493, + 484, + 521, + 545, + 457, + 490, + 470, + 471, + 522, + 494, + 483, + 468, + 487, + 490, + 472, + 470, + 471, + 484, + 488, + 500, + 496, + 465, + 483, + 477, + 499, + 465, + 446, + 495, + 472, + 468, + 461, + 513, + 493, + 467, + 481, + 488, + 481, + 507, + 503, + 494, + 500, + 487, + 492, + 516, + 484, + 433, + 524, + 507, + 522, + 506, + 482, + 464, + 503, + 460, + 502, + 511, + 490, + 517, + 520, + 523, + 520, + 478, + 486, + 511, + 508, + 487, + 495, + 511, + 466, + 478, + 504, + 485, + 480, + 483, + 482, + 469, + 488, + 513, + 517, + 501, + 452, + 482, + 466, + 471, + 460, + 476, + 478, + 514, + 487, + 473, + 507, + 520, + 487, + 484, + 484, + 487, + 468, + 465, + 439, + 469, + 485, + 494, + 488, + 508, + 477, + 495, + 465, + 525, + 502, + 501, + 475, + 487, + 456, + 512, + 515, + 476, + 444, + 470, + 526, + 521, + 550, + 517, + 523, + 456, + 465, + 486, + 477, + 475, + 479, + 481, + 513, + 550, + 505, + 488, + 467, + 518, + 493, + 485, + 452, + 506, + 483, + 495, + 497, + 500, + 465, + 502, + 500, + 468, + 480, + 491, + 487, + 461, + 490, + 452, + 467, + 462, + 472, + 510, + 477, + 494, + 481, + 500, + 476, + 494, + 485, + 496, + 520, + 474, + 455, + 484, + 517, + 492, + 506, + 498, + 495, + 518, + 466, + 467, + 487, + 511, + 506, + 523, + 481, + 453, + 475, + 472, + 501, + 511, + 480, + 509, + 475, + 466, + 479, + 464, + 492, + 485, + 455, + 490, + 478, + 482, + 461, + 481, + 501, + 506, + 479, + 498, + 485, + 451, + 510, + 475, + 482, + 473, + 459, + 443, + 496, + 449, + 447, + 459, + 451, + 527, + 469, + 509, + 464, + 450, + 476, + 496, + 494, + 493, + 479, + 532, + 505, + 487, + 425, + 462, + 484, + 475, + 478, + 510, + 486, + 509, + 479, + 498, + 522, + 552, + 449, + 507, + 531, + 514, + 454, + 476, + 496, + 482, + 494, + 479, + 519, + 500, + 495, + 536, + 447, + 487, + 533, + 524, + 495, + 466, + 475, + 547, + 525, + 485, + 499, + 477, + 457, + 515, + 447, + 489, + 435, + 480, + 485, + 492, + 483, + 496, + 458, + 447, + 475, + 491, + 487, + 481, + 489, + 482, + 487, + 489, + 485, + 485, + 458, + 480, + 474, + 496, + 523, + 491, + 457, + 488, + 503, + 490, + 525, + 451, + 502, + 481, + 499, + 478, + 491, + 510, + 487, + 500, + 493, + 458, + 485, + 504, + 500, + 455, + 519, + 478, + 477, + 486, + 481, + 488, + 499, + 492, + 510, + 489, + 487, + 493, + 452, + 505, + 464, + 489, + 476, + 463, + 483, + 471, + 468, + 475, + 447, + 468, + 448, + 502, + 481, + 489, + 512, + 507, + 451, + 475, + 502, + 484, + 496, + 517, + 482, + 499, + 453, + 515, + 481, + 474, + 518, + 493, + 462, + 481, + 507, + 507, + 476, + 474, + 484, + 487, + 484, + 537, + 508, + 552, + 486, + 476, + 500, + 481, + 458, + 508, + 510, + 474, + 469, + 518, + 449, + 462, + 511, + 466, + 459, + 507, + 474, + 448, + 467, + 484, + 519, + 478, + 481, + 501, + 507, + 485, + 483, + 489, + 477, + 517, + 504, + 511, + 468, + 506, + 445, + 529, + 507, + 518, + 467, + 494, + 493, + 464, + 499, + 488, + 484, + 490, + 515, + 525, + 492, + 463, + 494, + 454, + 528, + 455, + 479, + 470, + 474, + 491, + 509, + 479, + 477, + 450, + 485, + 492, + 494, + 468, + 454, + 513, + 488, + 463, + 425, + 486, + 498, + 517, + 457, + 521, + 459, + 508, + 526, + 490, + 500, + 457, + 450, + 484, + 504, + 511, + 512, + 447, + 542, + 430, + 496, + 482, + 485, + 501, + 487, + 486, + 477, + 471, + 541, + 456, + 484, + 514, + 503, + 488, + 480, + 489, + 484, + 470, + 468, + 467, + 491, + 466, + 492, + 516, + 479, + 510, + 522, + 485, + 473, + 488, + 477, + 482, + 448, + 475, + 528, + 476, + 459, + 529, + 492, + 501, + 479, + 507, + 483, + 435, + 486, + 537, + 455, + 496, + 489, + 466, + 508, + 494, + 507, + 502, + 483, + 501, + 498, + 552, + 463, + 519, + 460, + 496, + 456, + 475, + 519, + 491, + 506, + 502, + 491, + 498, + 435, + 500, + 514, + 484, + 490, + 513, + 492, + 461, + 505, + 488, + 455, + 465, + 512, + 495, + 438, + 481, + 494, + 501, + 472, + 461, + 527, + 521, + 505, + 479, + 512, + 516, + 485, + 506, + 493, + 460, + 447, + 469, + 502, + 472, + 470, + 499, + 483, + 477, + 465, + 502, + 509, + 492, + 468, + 504, + 457, + 519, + 477, + 503, + 485, + 479, + 508, + 481, + 517, + 477, + 480, + 474, + 473, + 477, + 463, + 471, + 504, + 495, + 481, + 490, + 515, + 472, + 478, + 446, + 483, + 507, + 452, + 461, + 504, + 491, + 514, + 479, + 516, + 479, + 476, + 491, + 491, + 494, + 513, + 505, + 491, + 473, + 518, + 512, + 497, + 515, + 442, + 497, + 513, + 453, + 482, + 471, + 549, + 459, + 507, + 516, + 460, + 496, + 503, + 495, + 474, + 493, + 464, + 553, + 486, + 497, + 468, + 473, + 508, + 551, + 547, + 438, + 467, + 483, + 488, + 463, + 479, + 500, + 517, + 497, + 498, + 498, + 491, + 456, + 472, + 525, + 533, + 479, + 491, + 480, + 450, + 505, + 463, + 487, + 451, + 495, + 512, + 466, + 498, + 498, + 510, + 523, + 496, + 478, + 468, + 513, + 508, + 512, + 465, + 509, + 500, + 500, + 488, + 504, + 459, + 495, + 463, + 515, + 480, + 510, + 495, + 468, + 487, + 514, + 487, + 479, + 447, + 492, + 490, + 508, + 457, + 476, + 503, + 485, + 486, + 535, + 462, + 510, + 474, + 475, + 484, + 487, + 515, + 485, + 493, + 502, + 456, + 486, + 543, + 509, + 500, + 461, + 504, + 503, + 447, + 488, + 445, + 458, + 458, + 508, + 466, + 477, + 479, + 503, + 453, + 511, + 535, + 532, + 497, + 490, + 495, + 496, + 472, + 451, + 502, + 482, + 479, + 468, + 464, + 484, + 537, + 439, + 474, + 482, + 502, + 472, + 492, + 517, + 488, + 481, + 506, + 485, + 486, + 459, + 468, + 478, + 446, + 464, + 506, + 434, + 558, + 491, + 514, + 474, + 482, + 472, + 471, + 484, + 523, + 473, + 467, + 518, + 516, + 443, + 457, + 513, + 451, + 483, + 485, + 501, + 491, + 490, + 480, + 457, + 493, + 493, + 518, + 521, + 485, + 481, + 504, + 495, + 491, + 483, + 514, + 500, + 460, + 469, + 514, + 484, + 480, + 515, + 498, + 494, + 496, + 506, + 487, + 480, + 483, + 503, + 496, + 492, + 432, + 442, + 482, + 514, + 492, + 485, + 488, + 467, + 480, + 458, + 482, + 529, + 528, + 456, + 528, + 485, + 517, + 457, + 501, + 490, + 471, + 455, + 513, + 460, + 502, + 505, + 498, + 476, + 508, + 484, + 500, + 487, + 499, + 478, + 473, + 483, + 500, + 496, + 485, + 506, + 450, + 487, + 537, + 470, + 461, + 486, + 492, + 511, + 473, + 482, + 511, + 484, + 494, + 492, + 485, + 498, + 511, + 477, + 488, + 487, + 474, + 500, + 454, + 515, + 477, + 507, + 505, + 485, + 527, + 456, + 493, + 484, + 509, + 526, + 473, + 458, + 493, + 524, + 511, + 497, + 481, + 504, + 484, + 486, + 489, + 477, + 515, + 489, + 520, + 506, + 475, + 517, + 465, + 490, + 504, + 480, + 461, + 504, + 524, + 477, + 465, + 507, + 471, + 486, + 503, + 453, + 481, + 493, + 490, + 482, + 495, + 508, + 482, + 431, + 487, + 503, + 479, + 495, + 493, + 501, + 515, + 516, + 469, + 460, + 448, + 530, + 468, + 510, + 472, + 491, + 464, + 501, + 502, + 483, + 508, + 499, + 494, + 456, + 425, + 489, + 548, + 484, + 489, + 505, + 481, + 497, + 495, + 465, + 490, + 475, + 494, + 505, + 530, + 479, + 506, + 489, + 529, + 462, + 502, + 478, + 487, + 464, + 471, + 477, + 496, + 479, + 462, + 500, + 507, + 481, + 489, + 509, + 519, + 504, + 511, + 470, + 428, + 482, + 526, + 502, + 465, + 448, + 460, + 449, + 509, + 480, + 491, + 493, + 485, + 461, + 474, + 432, + 488, + 508, + 500, + 451, + 468, + 473, + 501, + 495, + 454, + 510, + 478, + 506, + 511, + 517, + 463, + 477, + 466, + 517, + 478, + 464, + 477, + 493, + 526, + 489, + 471, + 511, + 488, + 499, + 512, + 484, + 494, + 544, + 482, + 487, + 492, + 506, + 520, + 484, + 493, + 485, + 504, + 512, + 469, + 478, + 480, + 504, + 511, + 487, + 509, + 480, + 502, + 464, + 468, + 486, + 484, + 493, + 532, + 441, + 520, + 499, + 523, + 467, + 462, + 488, + 482, + 470, + 511, + 532, + 490, + 515, + 490, + 487, + 479, + 485, + 505, + 504, + 530, + 434, + 472, + 488, + 485, + 506, + 456, + 500, + 533, + 467, + 513, + 455, + 442, + 470, + 521, + 479, + 492, + 477, + 490, + 501, + 478, + 503, + 469, + 476, + 440, + 482, + 484, + 471, + 511, + 492, + 513, + 524, + 493, + 505, + 466, + 515, + 495, + 466, + 466, + 490, + 471, + 481, + 468, + 492, + 519, + 457, + 464, + 507, + 528, + 468, + 465, + 517, + 475, + 488, + 481, + 490, + 472, + 430, + 477, + 516, + 498, + 508, + 489, + 487, + 522, + 478, + 473, + 538, + 533, + 497, + 475, + 482, + 521, + 504, + 503, + 494, + 503, + 467, + 467, + 475, + 462, + 495, + 526, + 495, + 478, + 491, + 459, + 493, + 491, + 534, + 531, + 478, + 502, + 490, + 473, + 477, + 475, + 476, + 494, + 505, + 487, + 462, + 494, + 468, + 502, + 433, + 523, + 450, + 495, + 455, + 479, + 506, + 488, + 500, + 499, + 498, + 463, + 531, + 488, + 477, + 455, + 480, + 480, + 514, + 507, + 483, + 495, + 521, + 491, + 497, + 510, + 524, + 463, + 468, + 501, + 450, + 492, + 456, + 498, + 471, + 500, + 491, + 497, + 501, + 479, + 502, + 526, + 484, + 461, + 487, + 533, + 464, + 468, + 494, + 487, + 465, + 471, + 506, + 510, + 451, + 501, + 478, + 458, + 496, + 502, + 492, + 512, + 482, + 469, + 491, + 519, + 519, + 491, + 518, + 507, + 463, + 473, + 487, + 507, + 442, + 484, + 446, + 450, + 454, + 498, + 482, + 487, + 492, + 484, + 548, + 494, + 470, + 458, + 503, + 481, + 492, + 488, + 531, + 487, + 450, + 468, + 440, + 476, + 460, + 474, + 497, + 526, + 518, + 489, + 501, + 524, + 503, + 464, + 464, + 537, + 489, + 469, + 500, + 525, + 469, + 479, + 491, + 450, + 523, + 444, + 487, + 458, + 472, + 514, + 473, + 493, + 480, + 486, + 517, + 510, + 458, + 468, + 480, + 467, + 513, + 530, + 505, + 461, + 436, + 491, + 472, + 479, + 531, + 493, + 468, + 464, + 540, + 493, + 502, + 496, + 490, + 490, + 462, + 513, + 506, + 504, + 497, + 508, + 503, + 477, + 500, + 512, + 490, + 480, + 496, + 467, + 517, + 473, + 476, + 486, + 505, + 498, + 532, + 513, + 452, + 498, + 448, + 508, + 526, + 488, + 488, + 488, + 479, + 542, + 493, + 507, + 496, + 491, + 493, + 482, + 441, + 462, + 503, + 512, + 475, + 500, + 508, + 496, + 506, + 470, + 477, + 511, + 480, + 452, + 453, + 489, + 473, + 507, + 488, + 473, + 494, + 492, + 489, + 491, + 457, + 481, + 493, + 494, + 451, + 503, + 489, + 475, + 477, + 473, + 494, + 499, + 499, + 475, + 478, + 488, + 489, + 475, + 451, + 512, + 525, + 514, + 504, + 480, + 482, + 486, + 481, + 480, + 497, + 465, + 503, + 473, + 465, + 487, + 468, + 481, + 499, + 450, + 496, + 521, + 447, + 462, + 485, + 519, + 498, + 491, + 462, + 489, + 492, + 485, + 473, + 519, + 503, + 540, + 539, + 482, + 482, + 531, + 532, + 461, + 473, + 542, + 503, + 532, + 420, + 490, + 453, + 491, + 471, + 477, + 470, + 490, + 485, + 499, + 482, + 500, + 517, + 489, + 500, + 511, + 452, + 453, + 486, + 452, + 502 + ] + }, + { + "counters": [ + 492, + 472, + 474, + 474, + 468, + 472, + 473, + 512, + 480, + 500, + 481, + 508, + 468, + 466, + 461, + 493, + 492, + 523, + 494, + 459, + 479, + 491, + 473, + 484, + 489, + 488, + 479, + 499, + 497, + 444, + 560, + 452, + 474, + 519, + 463, + 517, + 499, + 484, + 471, + 490, + 496, + 476, + 454, + 478, + 459, + 484, + 473, + 467, + 494, + 469, + 497, + 477, + 558, + 494, + 504, + 478, + 518, + 498, + 483, + 507, + 534, + 497, + 473, + 479, + 485, + 477, + 477, + 487, + 477, + 442, + 480, + 499, + 498, + 500, + 503, + 486, + 480, + 490, + 466, + 482, + 468, + 455, + 505, + 483, + 466, + 468, + 497, + 485, + 513, + 466, + 524, + 491, + 482, + 543, + 439, + 521, + 517, + 441, + 451, + 492, + 524, + 479, + 480, + 504, + 486, + 473, + 481, + 532, + 449, + 515, + 492, + 485, + 510, + 508, + 471, + 484, + 466, + 549, + 499, + 491, + 498, + 511, + 519, + 496, + 512, + 502, + 518, + 469, + 466, + 495, + 466, + 511, + 462, + 505, + 482, + 511, + 493, + 464, + 435, + 509, + 503, + 508, + 473, + 464, + 446, + 490, + 479, + 471, + 450, + 490, + 504, + 493, + 494, + 493, + 465, + 446, + 473, + 431, + 497, + 471, + 476, + 498, + 463, + 510, + 478, + 536, + 488, + 512, + 473, + 486, + 510, + 529, + 485, + 501, + 522, + 461, + 476, + 518, + 491, + 496, + 492, + 474, + 441, + 481, + 493, + 514, + 483, + 510, + 499, + 421, + 484, + 507, + 506, + 454, + 466, + 460, + 475, + 503, + 461, + 486, + 494, + 494, + 503, + 460, + 496, + 465, + 467, + 517, + 498, + 483, + 479, + 469, + 492, + 507, + 491, + 487, + 473, + 506, + 510, + 489, + 489, + 447, + 486, + 481, + 497, + 473, + 497, + 519, + 479, + 514, + 499, + 475, + 470, + 509, + 474, + 458, + 473, + 474, + 479, + 510, + 500, + 466, + 461, + 474, + 513, + 496, + 477, + 501, + 499, + 483, + 493, + 468, + 519, + 467, + 432, + 488, + 476, + 507, + 458, + 515, + 492, + 455, + 481, + 508, + 487, + 524, + 488, + 441, + 447, + 422, + 481, + 501, + 526, + 476, + 471, + 512, + 499, + 451, + 515, + 500, + 523, + 487, + 492, + 462, + 485, + 508, + 479, + 477, + 487, + 477, + 484, + 449, + 477, + 498, + 453, + 495, + 456, + 434, + 463, + 480, + 494, + 501, + 495, + 487, + 522, + 468, + 469, + 515, + 515, + 472, + 523, + 488, + 473, + 481, + 471, + 488, + 470, + 442, + 497, + 516, + 496, + 488, + 498, + 542, + 469, + 431, + 502, + 431, + 484, + 499, + 530, + 467, + 477, + 482, + 473, + 505, + 473, + 497, + 467, + 482, + 495, + 486, + 522, + 456, + 505, + 482, + 520, + 479, + 527, + 510, + 499, + 477, + 487, + 422, + 478, + 495, + 466, + 474, + 496, + 490, + 442, + 483, + 489, + 472, + 483, + 511, + 505, + 469, + 497, + 479, + 507, + 516, + 461, + 491, + 504, + 481, + 500, + 483, + 517, + 478, + 472, + 529, + 514, + 491, + 533, + 485, + 475, + 493, + 492, + 472, + 490, + 520, + 479, + 468, + 506, + 502, + 464, + 464, + 476, + 461, + 512, + 518, + 497, + 507, + 495, + 477, + 489, + 457, + 493, + 455, + 478, + 503, + 428, + 519, + 470, + 481, + 472, + 492, + 478, + 483, + 466, + 479, + 481, + 485, + 513, + 486, + 454, + 497, + 487, + 476, + 516, + 501, + 458, + 459, + 471, + 479, + 546, + 499, + 464, + 452, + 437, + 496, + 494, + 507, + 481, + 476, + 464, + 497, + 493, + 512, + 503, + 472, + 491, + 496, + 495, + 492, + 502, + 521, + 482, + 484, + 527, + 476, + 498, + 454, + 453, + 526, + 519, + 484, + 482, + 522, + 511, + 519, + 471, + 493, + 486, + 507, + 509, + 504, + 513, + 461, + 518, + 524, + 475, + 469, + 454, + 492, + 536, + 464, + 531, + 456, + 463, + 478, + 468, + 487, + 483, + 534, + 489, + 471, + 470, + 517, + 473, + 470, + 476, + 524, + 490, + 506, + 525, + 499, + 493, + 481, + 466, + 479, + 492, + 482, + 498, + 502, + 477, + 488, + 482, + 505, + 488, + 473, + 474, + 465, + 457, + 513, + 502, + 481, + 519, + 521, + 507, + 524, + 490, + 506, + 500, + 507, + 461, + 494, + 472, + 510, + 513, + 459, + 480, + 508, + 481, + 458, + 512, + 478, + 488, + 501, + 475, + 473, + 482, + 467, + 519, + 476, + 465, + 511, + 510, + 517, + 455, + 461, + 462, + 488, + 500, + 479, + 450, + 540, + 471, + 485, + 507, + 466, + 465, + 454, + 483, + 448, + 493, + 492, + 478, + 472, + 485, + 495, + 492, + 484, + 482, + 473, + 440, + 498, + 483, + 465, + 479, + 454, + 467, + 485, + 506, + 450, + 479, + 482, + 533, + 479, + 483, + 475, + 459, + 498, + 481, + 481, + 435, + 488, + 486, + 469, + 499, + 488, + 475, + 497, + 479, + 490, + 469, + 508, + 499, + 514, + 517, + 498, + 521, + 511, + 473, + 529, + 505, + 495, + 485, + 497, + 501, + 498, + 485, + 514, + 492, + 531, + 460, + 502, + 450, + 488, + 511, + 499, + 487, + 491, + 498, + 480, + 478, + 515, + 479, + 472, + 489, + 447, + 495, + 505, + 459, + 466, + 489, + 465, + 475, + 494, + 500, + 510, + 490, + 517, + 497, + 494, + 478, + 501, + 469, + 497, + 507, + 465, + 506, + 491, + 466, + 493, + 487, + 520, + 454, + 552, + 472, + 451, + 490, + 507, + 498, + 510, + 524, + 485, + 503, + 462, + 514, + 432, + 505, + 474, + 481, + 454, + 497, + 477, + 514, + 502, + 510, + 523, + 471, + 495, + 465, + 488, + 498, + 499, + 506, + 522, + 454, + 474, + 465, + 501, + 453, + 495, + 474, + 458, + 506, + 536, + 494, + 455, + 474, + 463, + 478, + 465, + 503, + 474, + 461, + 499, + 521, + 486, + 478, + 501, + 516, + 470, + 439, + 474, + 516, + 507, + 541, + 509, + 492, + 461, + 507, + 496, + 499, + 496, + 468, + 472, + 461, + 477, + 484, + 510, + 504, + 490, + 466, + 485, + 483, + 496, + 484, + 512, + 483, + 521, + 500, + 477, + 479, + 500, + 476, + 543, + 478, + 505, + 483, + 471, + 495, + 484, + 484, + 475, + 512, + 476, + 478, + 469, + 497, + 475, + 474, + 485, + 475, + 480, + 478, + 475, + 498, + 467, + 518, + 502, + 493, + 488, + 495, + 491, + 526, + 477, + 524, + 491, + 438, + 485, + 463, + 492, + 484, + 509, + 494, + 509, + 466, + 493, + 500, + 492, + 486, + 457, + 475, + 490, + 542, + 486, + 488, + 489, + 498, + 506, + 521, + 507, + 516, + 494, + 478, + 518, + 516, + 489, + 517, + 497, + 445, + 485, + 484, + 455, + 509, + 478, + 510, + 486, + 507, + 491, + 495, + 484, + 498, + 491, + 534, + 429, + 489, + 521, + 488, + 475, + 489, + 464, + 482, + 531, + 473, + 475, + 483, + 462, + 492, + 505, + 484, + 493, + 483, + 453, + 498, + 451, + 490, + 530, + 516, + 466, + 485, + 486, + 433, + 481, + 471, + 526, + 520, + 447, + 519, + 514, + 449, + 489, + 552, + 508, + 476, + 506, + 477, + 482, + 459, + 486, + 463, + 496, + 457, + 456, + 446, + 466, + 459, + 519, + 465, + 465, + 481, + 458, + 494, + 492, + 496, + 500, + 485, + 481, + 507, + 478, + 544, + 477, + 479, + 479, + 501, + 469, + 456, + 493, + 495, + 465, + 517, + 428, + 465, + 514, + 494, + 470, + 505, + 488, + 484, + 483, + 498, + 493, + 493, + 519, + 492, + 494, + 494, + 484, + 470, + 484, + 490, + 510, + 456, + 473, + 502, + 482, + 487, + 470, + 458, + 474, + 499, + 472, + 479, + 514, + 429, + 475, + 493, + 507, + 501, + 521, + 520, + 490, + 513, + 498, + 495, + 507, + 494, + 487, + 498, + 499, + 535, + 531, + 524, + 446, + 486, + 450, + 455, + 468, + 484, + 500, + 509, + 499, + 502, + 489, + 526, + 501, + 515, + 476, + 490, + 510, + 492, + 440, + 468, + 487, + 430, + 514, + 475, + 460, + 475, + 504, + 482, + 451, + 504, + 512, + 464, + 486, + 502, + 513, + 480, + 508, + 467, + 483, + 446, + 511, + 480, + 492, + 496, + 528, + 505, + 485, + 469, + 436, + 529, + 468, + 456, + 493, + 479, + 499, + 493, + 500, + 487, + 528, + 475, + 499, + 480, + 505, + 482, + 512, + 519, + 493, + 499, + 478, + 495, + 479, + 498, + 520, + 480, + 485, + 506, + 465, + 518, + 479, + 511, + 468, + 505, + 472, + 492, + 500, + 523, + 492, + 494, + 470, + 535, + 471, + 455, + 463, + 505, + 477, + 499, + 484, + 508, + 490, + 493, + 484, + 492, + 523, + 510, + 471, + 487, + 511, + 467, + 478, + 484, + 504, + 488, + 496, + 471, + 443, + 487, + 499, + 515, + 504, + 503, + 514, + 516, + 472, + 491, + 461, + 465, + 499, + 464, + 487, + 479, + 475, + 511, + 482, + 468, + 467, + 499, + 524, + 494, + 516, + 470, + 483, + 488, + 485, + 453, + 474, + 475, + 516, + 466, + 486, + 515, + 464, + 477, + 489, + 471, + 492, + 500, + 462, + 495, + 460, + 482, + 488, + 476, + 478, + 488, + 496, + 516, + 485, + 503, + 498, + 507, + 471, + 506, + 495, + 503, + 506, + 479, + 464, + 478, + 507, + 446, + 506, + 491, + 478, + 463, + 460, + 478, + 495, + 504, + 521, + 497, + 491, + 495, + 491, + 484, + 474, + 500, + 497, + 509, + 492, + 470, + 504, + 509, + 499, + 555, + 485, + 462, + 465, + 469, + 523, + 485, + 471, + 519, + 493, + 480, + 456, + 492, + 490, + 468, + 496, + 520, + 479, + 500, + 482, + 516, + 475, + 505, + 471, + 488, + 459, + 483, + 500, + 518, + 496, + 492, + 488, + 479, + 526, + 436, + 497, + 479, + 474, + 488, + 526, + 513, + 497, + 508, + 484, + 522, + 495, + 465, + 499, + 497, + 482, + 487, + 496, + 497, + 500, + 532, + 500, + 506, + 484, + 448, + 462, + 446, + 494, + 475, + 477, + 515, + 519, + 494, + 505, + 489, + 560, + 481, + 515, + 489, + 483, + 558, + 446, + 480, + 445, + 489, + 512, + 453, + 523, + 515, + 469, + 434, + 480, + 511, + 479, + 472, + 481, + 473, + 466, + 454, + 506, + 456, + 492, + 468, + 518, + 534, + 530, + 520, + 525, + 495, + 463, + 466, + 492, + 482, + 516, + 506, + 445, + 478, + 502, + 487, + 478, + 495, + 501, + 503, + 480, + 511, + 463, + 490, + 485, + 479, + 496, + 518, + 475, + 492, + 464, + 460, + 487, + 485, + 491, + 509, + 449, + 492, + 501, + 485, + 526, + 523, + 451, + 495, + 460, + 480, + 513, + 520, + 496, + 538, + 499, + 484, + 481, + 480, + 514, + 467, + 472, + 476, + 494, + 495, + 467, + 485, + 511, + 476, + 495, + 476, + 506, + 496, + 496, + 525, + 480, + 522, + 523, + 489, + 506, + 467, + 507, + 489, + 479, + 523, + 479, + 494, + 437, + 470, + 499, + 508, + 484, + 495, + 478, + 462, + 482, + 539, + 516, + 492, + 499, + 508, + 493, + 470, + 484, + 496, + 492, + 498, + 451, + 496, + 497, + 479, + 456, + 458, + 466, + 468, + 535, + 498, + 527, + 509, + 505, + 522, + 506, + 465, + 486, + 425, + 530, + 502, + 514, + 483, + 524, + 464, + 460, + 512, + 494, + 500, + 493, + 459, + 487, + 502, + 491, + 498, + 485, + 496, + 516, + 467, + 485, + 519, + 481, + 501, + 487, + 483, + 476, + 489, + 511, + 463, + 471, + 486, + 508, + 536, + 488, + 449, + 481, + 503, + 480, + 472, + 489, + 459, + 543, + 465, + 483, + 467, + 460, + 508, + 492, + 441, + 534, + 497, + 459, + 506, + 483, + 479, + 510, + 484, + 468, + 488, + 475, + 518, + 491, + 480, + 533, + 513, + 460, + 560, + 484, + 481, + 476, + 497, + 503, + 484, + 492, + 455, + 485, + 473, + 502, + 528, + 485, + 524, + 492, + 469, + 500, + 447, + 476, + 501, + 494, + 498, + 481, + 505, + 499, + 502, + 466, + 490, + 489, + 500, + 464, + 504, + 495, + 532, + 440, + 483, + 506, + 472, + 483, + 496, + 474, + 481, + 507, + 489, + 480, + 502, + 486, + 510, + 448, + 524, + 493, + 472, + 512, + 492, + 490, + 458, + 464, + 460, + 469, + 505, + 484, + 479, + 511, + 503, + 480, + 490, + 473, + 517, + 470, + 499, + 492, + 512, + 455, + 487, + 483, + 495, + 505, + 500, + 463, + 492, + 515, + 494, + 507, + 513, + 520, + 466, + 488, + 473, + 466, + 467, + 445, + 438, + 492, + 465, + 492, + 540, + 451, + 521, + 480, + 475, + 445, + 471, + 466, + 472, + 482, + 512, + 471, + 490, + 487, + 501, + 473, + 445, + 506, + 470, + 500, + 471, + 457, + 531, + 508, + 477, + 498, + 514, + 477, + 494, + 480, + 484, + 525, + 477, + 497, + 498, + 520, + 519, + 485, + 481, + 528, + 484, + 498, + 523, + 504, + 478, + 487, + 489, + 482, + 481, + 486, + 488, + 499, + 461, + 482, + 490, + 473, + 496, + 467, + 461, + 459, + 496, + 490, + 484, + 486, + 503, + 480, + 473, + 449, + 462, + 471, + 518, + 494, + 486, + 482, + 523, + 466, + 466, + 475, + 491, + 489, + 458, + 472, + 461, + 495, + 501, + 507, + 441, + 460, + 461, + 474, + 473, + 465, + 483, + 494, + 491, + 494, + 487, + 497, + 523, + 464, + 508, + 505, + 465, + 475, + 485, + 457, + 467, + 470, + 506, + 482, + 469, + 512, + 463, + 522, + 455, + 454, + 476, + 504, + 502, + 505, + 519, + 509, + 488, + 498, + 522, + 493, + 462, + 461, + 489, + 468, + 506, + 506, + 461, + 460, + 466, + 494, + 472, + 481, + 520, + 479, + 474, + 470, + 503, + 526, + 505, + 482, + 494, + 518, + 504, + 478, + 478, + 515, + 504, + 494, + 462, + 501, + 489, + 508, + 494, + 517, + 478, + 480, + 456, + 500, + 482, + 490, + 449, + 490, + 517, + 490, + 475, + 467, + 468, + 509, + 471, + 459, + 491, + 503, + 511, + 494, + 468, + 482, + 492, + 498, + 464, + 503, + 469, + 500, + 531, + 511, + 500, + 489, + 504, + 492, + 485, + 501, + 501, + 508, + 474, + 474, + 451, + 487, + 505, + 497, + 493, + 485, + 468, + 479, + 490, + 520, + 466, + 507, + 451, + 479, + 469, + 565, + 488, + 473, + 493, + 518, + 488, + 534, + 471, + 452, + 524, + 470, + 517, + 494, + 496, + 505, + 486, + 493, + 489, + 497, + 495, + 503, + 513, + 516, + 482, + 513, + 455, + 470, + 466, + 527, + 495, + 490, + 476, + 492, + 454, + 500, + 481, + 493, + 492, + 512, + 451, + 481, + 528, + 510, + 479, + 499, + 499, + 494, + 478, + 512, + 476, + 483, + 504, + 531, + 511, + 453, + 460, + 496, + 467, + 463, + 467, + 531, + 485, + 455, + 458, + 504, + 477, + 492, + 481, + 501, + 499, + 462, + 493, + 480, + 504, + 517, + 476, + 487, + 484, + 494, + 512, + 521, + 481, + 453, + 470, + 461, + 519, + 479, + 495, + 464, + 466, + 468, + 507, + 503, + 493, + 474, + 482, + 508, + 473, + 484, + 479, + 487, + 476, + 485, + 460, + 449, + 507, + 506, + 478, + 471, + 481, + 477, + 479, + 507, + 472, + 484, + 464, + 504, + 512, + 510, + 497, + 482, + 484, + 511, + 480, + 533, + 512, + 523, + 508, + 502, + 493, + 473, + 514, + 531, + 505, + 491, + 483, + 435, + 512, + 488, + 495, + 501, + 479, + 459, + 495, + 487, + 476, + 478, + 455, + 503, + 482, + 508, + 469, + 532, + 500, + 502, + 468, + 495, + 489, + 532, + 496, + 521, + 471, + 494, + 464, + 510, + 500, + 476, + 528, + 498, + 496, + 504, + 487, + 481, + 482, + 531, + 483, + 509, + 499, + 540, + 514, + 518, + 510, + 465, + 467, + 475, + 502, + 498, + 473, + 488, + 514, + 491, + 491, + 435, + 469, + 470, + 471, + 460, + 483, + 477, + 530, + 496, + 468, + 448, + 479, + 511, + 507, + 453, + 515, + 467, + 476, + 513, + 481, + 469, + 493, + 467, + 454, + 487, + 468, + 492, + 468, + 471, + 469, + 472, + 458, + 450, + 456, + 485, + 542, + 508, + 483, + 501, + 485, + 512, + 481, + 512, + 509, + 521, + 489, + 495, + 478, + 529, + 478, + 504, + 473, + 465, + 529, + 482, + 500, + 483, + 465, + 470, + 478, + 507, + 507, + 473, + 504, + 485, + 461, + 517, + 483, + 489, + 526, + 487, + 477, + 485, + 501, + 444, + 530, + 504, + 470, + 463, + 489, + 497 + ] + }, + { + "counters": [ + 486, + 470, + 490, + 485, + 472, + 508, + 468, + 498, + 490, + 487, + 516, + 485, + 486, + 519, + 497, + 503, + 474, + 490, + 491, + 477, + 483, + 473, + 474, + 478, + 483, + 459, + 503, + 478, + 524, + 464, + 472, + 486, + 492, + 449, + 455, + 486, + 458, + 477, + 491, + 482, + 493, + 505, + 498, + 506, + 516, + 450, + 499, + 455, + 469, + 466, + 542, + 548, + 519, + 482, + 487, + 479, + 491, + 477, + 505, + 465, + 478, + 460, + 452, + 488, + 470, + 481, + 522, + 519, + 471, + 470, + 528, + 499, + 518, + 511, + 493, + 521, + 489, + 477, + 478, + 519, + 502, + 511, + 490, + 466, + 494, + 446, + 503, + 468, + 468, + 467, + 489, + 504, + 474, + 508, + 500, + 449, + 486, + 490, + 530, + 467, + 481, + 485, + 511, + 488, + 521, + 470, + 508, + 525, + 510, + 504, + 511, + 502, + 513, + 461, + 501, + 485, + 476, + 504, + 489, + 486, + 488, + 497, + 541, + 482, + 487, + 494, + 482, + 499, + 506, + 452, + 484, + 495, + 470, + 453, + 482, + 489, + 494, + 485, + 475, + 491, + 481, + 498, + 523, + 493, + 483, + 505, + 493, + 503, + 475, + 491, + 517, + 502, + 473, + 505, + 456, + 509, + 486, + 484, + 489, + 496, + 485, + 500, + 521, + 470, + 540, + 482, + 470, + 471, + 504, + 445, + 487, + 470, + 488, + 472, + 470, + 500, + 511, + 484, + 458, + 489, + 503, + 484, + 543, + 502, + 481, + 486, + 494, + 514, + 473, + 491, + 497, + 502, + 482, + 482, + 472, + 512, + 503, + 494, + 531, + 517, + 470, + 534, + 491, + 476, + 482, + 503, + 478, + 469, + 495, + 508, + 487, + 490, + 494, + 476, + 493, + 490, + 464, + 539, + 471, + 498, + 489, + 435, + 459, + 464, + 453, + 498, + 523, + 494, + 518, + 512, + 447, + 451, + 492, + 516, + 496, + 484, + 478, + 512, + 484, + 469, + 518, + 505, + 482, + 493, + 434, + 465, + 503, + 499, + 454, + 502, + 489, + 485, + 521, + 507, + 499, + 483, + 475, + 507, + 504, + 529, + 502, + 500, + 463, + 514, + 517, + 495, + 448, + 517, + 496, + 488, + 532, + 464, + 449, + 509, + 454, + 501, + 500, + 500, + 488, + 503, + 451, + 506, + 481, + 445, + 438, + 485, + 500, + 474, + 499, + 487, + 504, + 474, + 512, + 507, + 470, + 495, + 477, + 465, + 474, + 514, + 492, + 508, + 465, + 477, + 478, + 426, + 505, + 485, + 478, + 501, + 477, + 486, + 446, + 461, + 499, + 483, + 501, + 484, + 483, + 502, + 502, + 464, + 500, + 464, + 479, + 459, + 560, + 467, + 481, + 505, + 496, + 498, + 413, + 512, + 451, + 508, + 443, + 485, + 442, + 480, + 552, + 476, + 487, + 546, + 495, + 518, + 450, + 502, + 502, + 492, + 519, + 497, + 485, + 464, + 445, + 496, + 454, + 501, + 486, + 510, + 501, + 534, + 495, + 466, + 466, + 492, + 482, + 489, + 517, + 437, + 497, + 493, + 423, + 481, + 468, + 481, + 506, + 439, + 439, + 450, + 493, + 489, + 524, + 476, + 512, + 472, + 475, + 504, + 521, + 481, + 500, + 522, + 514, + 486, + 495, + 533, + 464, + 513, + 440, + 499, + 469, + 495, + 454, + 479, + 549, + 514, + 481, + 503, + 468, + 477, + 476, + 492, + 465, + 528, + 493, + 499, + 480, + 491, + 459, + 470, + 476, + 476, + 492, + 474, + 517, + 504, + 466, + 463, + 517, + 453, + 471, + 456, + 508, + 482, + 465, + 475, + 548, + 485, + 483, + 482, + 500, + 549, + 444, + 504, + 481, + 441, + 477, + 496, + 515, + 490, + 532, + 490, + 489, + 444, + 485, + 473, + 453, + 498, + 534, + 467, + 486, + 517, + 499, + 488, + 492, + 499, + 461, + 478, + 484, + 515, + 515, + 486, + 497, + 464, + 523, + 483, + 468, + 491, + 513, + 525, + 489, + 476, + 494, + 486, + 543, + 475, + 474, + 470, + 495, + 466, + 515, + 503, + 514, + 520, + 485, + 485, + 469, + 518, + 493, + 486, + 505, + 457, + 484, + 476, + 502, + 511, + 527, + 481, + 510, + 492, + 525, + 502, + 433, + 490, + 471, + 481, + 527, + 502, + 496, + 470, + 513, + 505, + 458, + 477, + 445, + 545, + 453, + 479, + 484, + 483, + 498, + 494, + 499, + 447, + 486, + 482, + 493, + 474, + 483, + 499, + 454, + 485, + 455, + 473, + 522, + 533, + 463, + 463, + 481, + 495, + 491, + 482, + 528, + 485, + 434, + 450, + 473, + 482, + 501, + 492, + 518, + 468, + 498, + 513, + 498, + 472, + 486, + 531, + 487, + 466, + 501, + 459, + 481, + 497, + 491, + 465, + 494, + 493, + 524, + 490, + 508, + 509, + 437, + 489, + 505, + 480, + 488, + 486, + 501, + 460, + 515, + 472, + 470, + 476, + 442, + 480, + 485, + 504, + 477, + 494, + 473, + 475, + 513, + 506, + 457, + 481, + 482, + 472, + 496, + 475, + 490, + 486, + 470, + 497, + 515, + 471, + 446, + 463, + 518, + 490, + 461, + 509, + 477, + 441, + 486, + 475, + 444, + 501, + 486, + 471, + 513, + 496, + 484, + 511, + 498, + 487, + 478, + 496, + 485, + 443, + 469, + 483, + 460, + 484, + 492, + 493, + 487, + 483, + 527, + 459, + 437, + 496, + 492, + 475, + 457, + 503, + 469, + 509, + 481, + 489, + 465, + 472, + 472, + 488, + 482, + 480, + 512, + 466, + 456, + 458, + 490, + 498, + 488, + 500, + 447, + 496, + 513, + 482, + 494, + 488, + 501, + 495, + 466, + 483, + 491, + 475, + 490, + 505, + 527, + 448, + 501, + 515, + 416, + 493, + 491, + 481, + 475, + 484, + 445, + 486, + 504, + 499, + 545, + 488, + 497, + 469, + 479, + 456, + 508, + 477, + 481, + 538, + 503, + 498, + 444, + 508, + 484, + 490, + 484, + 497, + 509, + 458, + 497, + 458, + 488, + 494, + 456, + 491, + 491, + 489, + 490, + 505, + 520, + 511, + 490, + 458, + 477, + 471, + 481, + 492, + 482, + 504, + 526, + 503, + 507, + 483, + 491, + 504, + 487, + 499, + 551, + 476, + 513, + 467, + 483, + 497, + 529, + 521, + 455, + 473, + 500, + 415, + 474, + 559, + 518, + 512, + 483, + 502, + 462, + 547, + 514, + 503, + 476, + 491, + 488, + 471, + 488, + 451, + 509, + 486, + 503, + 483, + 450, + 461, + 486, + 478, + 508, + 501, + 520, + 513, + 499, + 493, + 467, + 494, + 492, + 506, + 481, + 474, + 498, + 499, + 508, + 481, + 503, + 490, + 484, + 472, + 486, + 505, + 469, + 506, + 499, + 489, + 454, + 487, + 526, + 465, + 458, + 504, + 464, + 517, + 490, + 465, + 503, + 488, + 482, + 495, + 526, + 489, + 477, + 507, + 494, + 525, + 505, + 496, + 491, + 536, + 469, + 506, + 438, + 482, + 517, + 464, + 472, + 507, + 499, + 447, + 465, + 460, + 466, + 484, + 476, + 467, + 484, + 474, + 462, + 516, + 471, + 500, + 511, + 517, + 460, + 512, + 484, + 461, + 458, + 519, + 481, + 505, + 472, + 475, + 488, + 543, + 539, + 511, + 489, + 450, + 516, + 492, + 500, + 452, + 500, + 521, + 519, + 470, + 461, + 467, + 458, + 501, + 496, + 482, + 456, + 500, + 496, + 504, + 481, + 499, + 480, + 477, + 559, + 499, + 522, + 491, + 482, + 480, + 493, + 513, + 485, + 463, + 460, + 501, + 507, + 480, + 503, + 511, + 484, + 461, + 443, + 477, + 464, + 486, + 523, + 493, + 465, + 489, + 510, + 467, + 471, + 498, + 482, + 450, + 483, + 498, + 520, + 484, + 509, + 512, + 454, + 461, + 497, + 465, + 466, + 510, + 451, + 496, + 489, + 478, + 516, + 529, + 492, + 490, + 476, + 505, + 487, + 523, + 489, + 495, + 509, + 475, + 501, + 472, + 479, + 501, + 493, + 475, + 510, + 511, + 462, + 472, + 487, + 487, + 516, + 474, + 512, + 498, + 451, + 484, + 493, + 497, + 544, + 461, + 475, + 525, + 487, + 465, + 490, + 493, + 463, + 477, + 525, + 486, + 484, + 494, + 485, + 490, + 521, + 495, + 494, + 518, + 487, + 501, + 468, + 492, + 449, + 461, + 480, + 500, + 497, + 470, + 479, + 521, + 490, + 520, + 508, + 508, + 495, + 481, + 513, + 502, + 516, + 480, + 471, + 483, + 496, + 487, + 497, + 475, + 492, + 513, + 491, + 550, + 487, + 514, + 510, + 513, + 494, + 483, + 479, + 518, + 468, + 485, + 484, + 469, + 494, + 495, + 527, + 487, + 503, + 486, + 468, + 508, + 528, + 478, + 480, + 463, + 508, + 491, + 454, + 472, + 458, + 500, + 488, + 469, + 500, + 489, + 483, + 476, + 496, + 454, + 466, + 486, + 501, + 470, + 509, + 482, + 486, + 494, + 490, + 463, + 463, + 496, + 525, + 489, + 476, + 490, + 513, + 491, + 500, + 545, + 481, + 492, + 500, + 501, + 483, + 467, + 515, + 499, + 457, + 494, + 478, + 495, + 504, + 472, + 494, + 504, + 532, + 456, + 458, + 480, + 477, + 504, + 484, + 494, + 473, + 488, + 466, + 450, + 475, + 484, + 481, + 478, + 500, + 492, + 466, + 505, + 480, + 497, + 462, + 486, + 480, + 463, + 494, + 437, + 477, + 468, + 469, + 510, + 502, + 509, + 499, + 465, + 530, + 511, + 485, + 465, + 489, + 490, + 490, + 490, + 491, + 509, + 514, + 481, + 481, + 498, + 491, + 493, + 514, + 453, + 505, + 480, + 492, + 471, + 498, + 476, + 496, + 461, + 520, + 469, + 467, + 493, + 453, + 495, + 489, + 507, + 442, + 495, + 468, + 462, + 512, + 505, + 477, + 466, + 453, + 475, + 484, + 498, + 509, + 537, + 479, + 480, + 478, + 470, + 465, + 495, + 464, + 485, + 447, + 508, + 478, + 491, + 494, + 471, + 477, + 514, + 453, + 454, + 517, + 526, + 489, + 491, + 457, + 519, + 483, + 522, + 479, + 522, + 514, + 491, + 491, + 494, + 482, + 463, + 503, + 491, + 471, + 508, + 476, + 472, + 500, + 514, + 500, + 458, + 500, + 474, + 497, + 497, + 514, + 526, + 501, + 493, + 519, + 516, + 474, + 478, + 524, + 467, + 473, + 500, + 486, + 490, + 491, + 487, + 468, + 501, + 540, + 493, + 482, + 492, + 502, + 474, + 486, + 512, + 493, + 483, + 457, + 487, + 486, + 502, + 460, + 459, + 466, + 495, + 469, + 525, + 518, + 518, + 517, + 469, + 463, + 459, + 511, + 495, + 490, + 452, + 511, + 486, + 515, + 482, + 464, + 487, + 513, + 479, + 484, + 468, + 499, + 482, + 496, + 494, + 505, + 475, + 470, + 471, + 497, + 464, + 520, + 506, + 518, + 477, + 498, + 465, + 462, + 478, + 496, + 495, + 490, + 486, + 516, + 503, + 483, + 489, + 508, + 481, + 479, + 533, + 491, + 477, + 482, + 514, + 485, + 488, + 531, + 522, + 481, + 480, + 498, + 485, + 490, + 489, + 510, + 512, + 497, + 517, + 499, + 465, + 453, + 467, + 474, + 507, + 496, + 485, + 497, + 492, + 488, + 505, + 464, + 458, + 509, + 479, + 466, + 490, + 535, + 484, + 497, + 469, + 493, + 498, + 518, + 483, + 449, + 480, + 473, + 523, + 504, + 441, + 503, + 451, + 509, + 497, + 509, + 497, + 511, + 470, + 476, + 499, + 517, + 511, + 463, + 504, + 498, + 476, + 480, + 501, + 513, + 507, + 516, + 461, + 486, + 482, + 504, + 466, + 498, + 457, + 487, + 462, + 475, + 471, + 481, + 493, + 475, + 483, + 473, + 473, + 472, + 508, + 453, + 480, + 459, + 491, + 477, + 504, + 497, + 503, + 504, + 488, + 448, + 528, + 509, + 506, + 477, + 502, + 492, + 519, + 488, + 503, + 503, + 510, + 472, + 498, + 503, + 511, + 512, + 481, + 502, + 464, + 510, + 524, + 435, + 499, + 549, + 463, + 479, + 498, + 438, + 492, + 482, + 512, + 497, + 498, + 494, + 494, + 468, + 508, + 510, + 513, + 456, + 476, + 485, + 517, + 479, + 501, + 466, + 492, + 530, + 470, + 493, + 488, + 498, + 506, + 478, + 478, + 476, + 489, + 496, + 490, + 494, + 493, + 478, + 503, + 482, + 475, + 475, + 444, + 482, + 502, + 476, + 463, + 490, + 455, + 486, + 503, + 504, + 476, + 514, + 514, + 503, + 469, + 463, + 512, + 460, + 467, + 461, + 495, + 469, + 471, + 528, + 458, + 511, + 463, + 514, + 472, + 462, + 469, + 458, + 477, + 476, + 492, + 507, + 510, + 425, + 484, + 457, + 475, + 483, + 498, + 528, + 498, + 503, + 501, + 475, + 451, + 525, + 495, + 482, + 454, + 487, + 481, + 462, + 458, + 473, + 451, + 482, + 483, + 486, + 464, + 522, + 483, + 491, + 493, + 503, + 488, + 505, + 525, + 460, + 520, + 488, + 502, + 525, + 492, + 485, + 454, + 498, + 501, + 457, + 535, + 474, + 527, + 484, + 504, + 476, + 491, + 478, + 494, + 502, + 507, + 484, + 472, + 517, + 457, + 479, + 510, + 529, + 485, + 481, + 482, + 501, + 498, + 516, + 462, + 478, + 505, + 516, + 472, + 511, + 486, + 489, + 485, + 493, + 538, + 486, + 496, + 499, + 483, + 513, + 501, + 465, + 500, + 508, + 481, + 485, + 494, + 481, + 488, + 495, + 506, + 456, + 541, + 473, + 488, + 449, + 528, + 481, + 488, + 457, + 476, + 495, + 461, + 517, + 522, + 452, + 472, + 555, + 502, + 497, + 505, + 512, + 506, + 474, + 463, + 463, + 500, + 452, + 488, + 465, + 475, + 510, + 492, + 454, + 514, + 502, + 482, + 473, + 512, + 454, + 492, + 512, + 468, + 483, + 492, + 465, + 466, + 504, + 514, + 478, + 483, + 473, + 509, + 494, + 508, + 521, + 448, + 472, + 461, + 470, + 452, + 504, + 501, + 489, + 495, + 498, + 490, + 525, + 494, + 490, + 532, + 490, + 506, + 540, + 496, + 489, + 462, + 499, + 496, + 510, + 489, + 461, + 486, + 477, + 474, + 492, + 464, + 498, + 479, + 506, + 472, + 501, + 466, + 480, + 454, + 462, + 478, + 542, + 449, + 472, + 506, + 491, + 509, + 459, + 488, + 511, + 479, + 540, + 475, + 520, + 506, + 515, + 493, + 475, + 464, + 491, + 508, + 462, + 475, + 502, + 498, + 480, + 458, + 482, + 505, + 471, + 501, + 498, + 474, + 496, + 473, + 477, + 515, + 533, + 483, + 482, + 484, + 506, + 519, + 514, + 480, + 483, + 493, + 508, + 492, + 468, + 483, + 468, + 496, + 469, + 489, + 524, + 526, + 477, + 484, + 456, + 479, + 464, + 477, + 479, + 511, + 481, + 471, + 481, + 483, + 498, + 488, + 462, + 489, + 455, + 480, + 500, + 443, + 438, + 465, + 465, + 495, + 452, + 521, + 467, + 462, + 483, + 494, + 521, + 464, + 461, + 463, + 496, + 476, + 487, + 463, + 479, + 489, + 449, + 473, + 480, + 478, + 463, + 473, + 493, + 492, + 500, + 469, + 493, + 486, + 489, + 515, + 506, + 454, + 508, + 481, + 465, + 499, + 477, + 522, + 461, + 483, + 473, + 449, + 495, + 516, + 488, + 470, + 503, + 489, + 500, + 469, + 509, + 482, + 500, + 509, + 512, + 485, + 486, + 523, + 486, + 493, + 503, + 495, + 493, + 451, + 491, + 529, + 504, + 472, + 481, + 493, + 466, + 472, + 496, + 497, + 494, + 489, + 490, + 512, + 495, + 482, + 491, + 495, + 448, + 495, + 454, + 483, + 482, + 482, + 506, + 525, + 494, + 497, + 530, + 460, + 506, + 539, + 505, + 488, + 480, + 474, + 480, + 470, + 486, + 486, + 503, + 496, + 489, + 469, + 462, + 457, + 500, + 449, + 442, + 476, + 448, + 508, + 498, + 507, + 465, + 481, + 476, + 465, + 476, + 517, + 477, + 453, + 490, + 502, + 478, + 479, + 454, + 459, + 482, + 492, + 506, + 441, + 492, + 481, + 482, + 478, + 449, + 478, + 503, + 505, + 474, + 524, + 483, + 503, + 492, + 496, + 479, + 496, + 464, + 492, + 506, + 490, + 494, + 483, + 500, + 531, + 486, + 469, + 517, + 502, + 488, + 487, + 497, + 433, + 462, + 516, + 493, + 474, + 500, + 516, + 497, + 475, + 497, + 487, + 481, + 470, + 469, + 499, + 450, + 501, + 494, + 538, + 490, + 511, + 491, + 504, + 481, + 468, + 499, + 473, + 479, + 471, + 499, + 494, + 488, + 475, + 493, + 506, + 478, + 478, + 502, + 461, + 474, + 490, + 520, + 476, + 473, + 451, + 485, + 454, + 475, + 497, + 509, + 474, + 516, + 473, + 464, + 497, + 497, + 458, + 458, + 481, + 453, + 514, + 521, + 513, + 511, + 490, + 486, + 507, + 498, + 504, + 509, + 485, + 502, + 491, + 481, + 478, + 480, + 448, + 499, + 502, + 499, + 522 + ] + }, + { + "counters": [ + 482, + 510, + 472, + 488, + 524, + 444, + 511, + 511, + 477, + 475, + 538, + 524, + 512, + 483, + 549, + 475, + 515, + 495, + 511, + 461, + 529, + 510, + 501, + 471, + 469, + 488, + 505, + 489, + 474, + 472, + 463, + 499, + 477, + 505, + 501, + 431, + 492, + 522, + 512, + 458, + 456, + 534, + 493, + 501, + 512, + 549, + 504, + 491, + 482, + 492, + 451, + 465, + 482, + 477, + 477, + 496, + 484, + 475, + 470, + 472, + 460, + 504, + 461, + 495, + 518, + 501, + 514, + 480, + 465, + 517, + 491, + 479, + 495, + 443, + 487, + 508, + 462, + 467, + 483, + 452, + 473, + 479, + 532, + 497, + 491, + 501, + 464, + 497, + 493, + 483, + 485, + 471, + 532, + 492, + 453, + 505, + 470, + 502, + 499, + 506, + 464, + 465, + 516, + 506, + 479, + 460, + 500, + 493, + 542, + 468, + 485, + 480, + 469, + 468, + 537, + 477, + 467, + 502, + 448, + 499, + 491, + 497, + 467, + 510, + 450, + 487, + 493, + 457, + 488, + 469, + 462, + 488, + 485, + 538, + 518, + 496, + 498, + 478, + 499, + 511, + 493, + 504, + 463, + 489, + 463, + 467, + 491, + 498, + 521, + 482, + 454, + 458, + 517, + 500, + 447, + 455, + 467, + 493, + 481, + 457, + 482, + 469, + 529, + 510, + 481, + 486, + 435, + 498, + 545, + 489, + 482, + 450, + 484, + 496, + 501, + 465, + 471, + 476, + 493, + 499, + 486, + 503, + 458, + 502, + 482, + 479, + 477, + 505, + 455, + 493, + 468, + 438, + 461, + 480, + 458, + 484, + 497, + 494, + 506, + 460, + 471, + 498, + 532, + 470, + 507, + 478, + 456, + 476, + 471, + 512, + 487, + 490, + 535, + 494, + 518, + 515, + 478, + 496, + 515, + 478, + 477, + 475, + 481, + 489, + 516, + 455, + 512, + 508, + 503, + 503, + 483, + 530, + 434, + 475, + 500, + 467, + 512, + 533, + 530, + 507, + 471, + 509, + 459, + 494, + 501, + 478, + 475, + 486, + 458, + 498, + 452, + 469, + 452, + 533, + 473, + 496, + 492, + 476, + 478, + 484, + 474, + 506, + 464, + 509, + 486, + 481, + 488, + 486, + 486, + 486, + 498, + 490, + 485, + 467, + 477, + 513, + 500, + 492, + 489, + 448, + 499, + 479, + 519, + 479, + 477, + 499, + 503, + 497, + 468, + 496, + 513, + 459, + 476, + 477, + 493, + 478, + 496, + 494, + 478, + 507, + 489, + 500, + 448, + 507, + 507, + 506, + 499, + 504, + 507, + 511, + 445, + 498, + 463, + 491, + 500, + 485, + 513, + 511, + 517, + 458, + 484, + 510, + 481, + 517, + 501, + 524, + 509, + 482, + 465, + 482, + 506, + 482, + 474, + 528, + 493, + 463, + 465, + 487, + 459, + 463, + 527, + 533, + 490, + 510, + 472, + 507, + 480, + 494, + 454, + 471, + 516, + 450, + 507, + 475, + 469, + 454, + 486, + 505, + 492, + 535, + 516, + 542, + 478, + 448, + 465, + 471, + 475, + 522, + 505, + 475, + 488, + 469, + 468, + 465, + 487, + 511, + 470, + 525, + 520, + 498, + 463, + 505, + 498, + 491, + 476, + 493, + 503, + 512, + 483, + 512, + 477, + 492, + 493, + 462, + 515, + 471, + 469, + 484, + 523, + 504, + 507, + 466, + 515, + 486, + 464, + 478, + 478, + 497, + 471, + 486, + 472, + 513, + 498, + 465, + 477, + 504, + 493, + 481, + 507, + 523, + 497, + 456, + 493, + 486, + 504, + 481, + 470, + 492, + 513, + 490, + 489, + 481, + 510, + 477, + 496, + 516, + 468, + 512, + 452, + 477, + 479, + 463, + 459, + 469, + 461, + 465, + 480, + 489, + 515, + 485, + 500, + 477, + 505, + 452, + 509, + 526, + 511, + 513, + 471, + 452, + 492, + 501, + 514, + 442, + 520, + 506, + 486, + 495, + 525, + 461, + 462, + 452, + 493, + 479, + 492, + 534, + 501, + 501, + 496, + 461, + 469, + 502, + 502, + 457, + 455, + 451, + 449, + 503, + 475, + 473, + 531, + 437, + 464, + 460, + 511, + 464, + 478, + 493, + 490, + 539, + 488, + 478, + 481, + 510, + 468, + 464, + 480, + 538, + 507, + 502, + 479, + 498, + 478, + 452, + 523, + 517, + 509, + 483, + 478, + 466, + 489, + 504, + 506, + 505, + 495, + 475, + 476, + 520, + 489, + 504, + 496, + 501, + 508, + 480, + 511, + 460, + 491, + 475, + 479, + 491, + 510, + 537, + 509, + 491, + 500, + 477, + 466, + 481, + 463, + 479, + 472, + 478, + 509, + 499, + 458, + 520, + 482, + 506, + 505, + 486, + 469, + 501, + 479, + 476, + 466, + 450, + 474, + 462, + 479, + 500, + 487, + 472, + 462, + 473, + 481, + 492, + 489, + 464, + 465, + 478, + 498, + 505, + 501, + 493, + 502, + 483, + 487, + 490, + 503, + 501, + 501, + 495, + 470, + 494, + 528, + 483, + 477, + 518, + 455, + 482, + 487, + 448, + 514, + 491, + 495, + 484, + 448, + 507, + 488, + 529, + 493, + 466, + 493, + 480, + 481, + 495, + 448, + 501, + 504, + 492, + 479, + 488, + 523, + 515, + 497, + 469, + 497, + 494, + 486, + 500, + 467, + 539, + 503, + 479, + 504, + 504, + 482, + 481, + 494, + 503, + 482, + 527, + 470, + 478, + 505, + 506, + 476, + 487, + 496, + 498, + 499, + 478, + 513, + 491, + 463, + 504, + 467, + 510, + 518, + 512, + 495, + 484, + 504, + 473, + 480, + 478, + 476, + 483, + 490, + 475, + 468, + 457, + 499, + 492, + 464, + 505, + 492, + 488, + 492, + 502, + 512, + 492, + 482, + 528, + 493, + 488, + 491, + 506, + 458, + 516, + 455, + 468, + 510, + 473, + 515, + 481, + 496, + 519, + 465, + 487, + 457, + 517, + 452, + 483, + 478, + 505, + 501, + 511, + 487, + 492, + 488, + 477, + 500, + 481, + 501, + 505, + 466, + 448, + 477, + 470, + 531, + 496, + 494, + 466, + 467, + 465, + 527, + 499, + 439, + 481, + 484, + 472, + 492, + 514, + 508, + 484, + 508, + 480, + 477, + 477, + 460, + 470, + 474, + 495, + 450, + 463, + 520, + 490, + 533, + 483, + 456, + 504, + 491, + 480, + 445, + 486, + 488, + 498, + 473, + 471, + 489, + 462, + 516, + 505, + 478, + 456, + 463, + 509, + 541, + 478, + 465, + 491, + 465, + 522, + 474, + 523, + 516, + 459, + 472, + 491, + 480, + 515, + 521, + 459, + 480, + 528, + 484, + 447, + 443, + 477, + 482, + 463, + 532, + 462, + 498, + 496, + 468, + 480, + 457, + 478, + 524, + 485, + 445, + 465, + 515, + 488, + 504, + 475, + 460, + 481, + 500, + 470, + 493, + 494, + 504, + 483, + 495, + 455, + 502, + 517, + 498, + 472, + 498, + 491, + 496, + 512, + 535, + 515, + 500, + 499, + 468, + 487, + 494, + 486, + 517, + 465, + 470, + 477, + 474, + 557, + 489, + 503, + 478, + 498, + 508, + 485, + 474, + 487, + 466, + 494, + 482, + 487, + 486, + 468, + 521, + 498, + 509, + 507, + 514, + 508, + 497, + 499, + 498, + 487, + 458, + 462, + 496, + 530, + 452, + 488, + 499, + 470, + 507, + 487, + 468, + 455, + 506, + 490, + 465, + 449, + 468, + 498, + 484, + 440, + 499, + 506, + 457, + 492, + 496, + 499, + 452, + 480, + 496, + 484, + 510, + 445, + 471, + 496, + 488, + 501, + 490, + 468, + 500, + 469, + 479, + 478, + 447, + 501, + 502, + 508, + 484, + 508, + 506, + 516, + 500, + 473, + 490, + 482, + 472, + 492, + 510, + 509, + 465, + 506, + 493, + 483, + 506, + 500, + 480, + 495, + 489, + 460, + 471, + 517, + 500, + 487, + 477, + 490, + 483, + 501, + 487, + 502, + 491, + 501, + 457, + 497, + 484, + 495, + 505, + 482, + 436, + 459, + 489, + 508, + 465, + 465, + 472, + 491, + 498, + 457, + 486, + 502, + 457, + 488, + 476, + 469, + 497, + 496, + 476, + 482, + 467, + 528, + 514, + 479, + 466, + 445, + 484, + 477, + 507, + 446, + 449, + 485, + 498, + 517, + 518, + 468, + 472, + 453, + 519, + 508, + 495, + 487, + 493, + 530, + 484, + 492, + 481, + 508, + 485, + 454, + 477, + 492, + 487, + 467, + 451, + 474, + 491, + 496, + 496, + 524, + 472, + 494, + 487, + 495, + 467, + 516, + 480, + 488, + 472, + 474, + 502, + 515, + 447, + 475, + 472, + 498, + 506, + 492, + 471, + 520, + 478, + 465, + 462, + 524, + 514, + 496, + 518, + 501, + 481, + 468, + 509, + 475, + 512, + 498, + 502, + 501, + 466, + 534, + 479, + 501, + 535, + 462, + 452, + 517, + 496, + 472, + 500, + 466, + 496, + 480, + 510, + 491, + 495, + 445, + 575, + 489, + 476, + 473, + 506, + 493, + 502, + 447, + 530, + 494, + 486, + 484, + 524, + 481, + 467, + 488, + 524, + 511, + 504, + 496, + 462, + 542, + 495, + 533, + 498, + 507, + 466, + 500, + 481, + 487, + 476, + 514, + 480, + 497, + 527, + 497, + 472, + 477, + 490, + 507, + 478, + 492, + 475, + 493, + 464, + 488, + 461, + 510, + 503, + 453, + 511, + 480, + 515, + 513, + 464, + 466, + 472, + 479, + 467, + 485, + 499, + 467, + 505, + 497, + 518, + 486, + 485, + 463, + 494, + 510, + 521, + 437, + 464, + 481, + 485, + 505, + 546, + 481, + 476, + 493, + 488, + 481, + 525, + 504, + 484, + 529, + 514, + 467, + 448, + 485, + 476, + 497, + 428, + 492, + 492, + 474, + 500, + 479, + 526, + 467, + 478, + 479, + 489, + 471, + 486, + 465, + 494, + 455, + 447, + 452, + 475, + 500, + 521, + 516, + 482, + 500, + 463, + 459, + 496, + 501, + 488, + 479, + 463, + 498, + 493, + 495, + 483, + 507, + 510, + 453, + 482, + 494, + 485, + 479, + 485, + 518, + 490, + 506, + 483, + 477, + 496, + 468, + 507, + 514, + 491, + 482, + 481, + 498, + 488, + 459, + 459, + 483, + 492, + 494, + 508, + 521, + 457, + 513, + 471, + 489, + 494, + 462, + 496, + 490, + 457, + 500, + 455, + 484, + 500, + 514, + 509, + 511, + 469, + 480, + 477, + 476, + 493, + 472, + 474, + 468, + 498, + 509, + 475, + 484, + 504, + 502, + 492, + 547, + 489, + 477, + 459, + 477, + 508, + 499, + 472, + 473, + 493, + 509, + 469, + 520, + 481, + 436, + 474, + 530, + 479, + 470, + 504, + 488, + 490, + 484, + 457, + 447, + 486, + 491, + 532, + 488, + 487, + 496, + 490, + 513, + 517, + 498, + 489, + 503, + 491, + 514, + 499, + 494, + 497, + 504, + 454, + 516, + 464, + 483, + 482, + 510, + 435, + 467, + 481, + 465, + 458, + 463, + 473, + 456, + 491, + 487, + 454, + 485, + 478, + 476, + 470, + 474, + 473, + 462, + 492, + 505, + 519, + 536, + 488, + 496, + 498, + 466, + 509, + 485, + 498, + 479, + 518, + 504, + 478, + 498, + 505, + 483, + 485, + 495, + 516, + 462, + 514, + 452, + 492, + 510, + 462, + 457, + 502, + 468, + 499, + 551, + 519, + 505, + 488, + 473, + 529, + 489, + 497, + 486, + 526, + 455, + 485, + 483, + 465, + 533, + 494, + 518, + 485, + 516, + 494, + 477, + 473, + 471, + 496, + 499, + 458, + 473, + 482, + 488, + 459, + 430, + 474, + 457, + 512, + 478, + 489, + 467, + 472, + 495, + 473, + 460, + 512, + 475, + 514, + 450, + 465, + 515, + 527, + 486, + 499, + 503, + 486, + 530, + 466, + 474, + 483, + 511, + 511, + 490, + 489, + 483, + 514, + 476, + 469, + 558, + 501, + 513, + 469, + 484, + 491, + 486, + 494, + 502, + 509, + 485, + 479, + 486, + 472, + 448, + 492, + 536, + 521, + 492, + 483, + 494, + 473, + 506, + 496, + 538, + 439, + 540, + 547, + 503, + 497, + 496, + 486, + 491, + 477, + 469, + 524, + 462, + 478, + 492, + 470, + 502, + 491, + 464, + 486, + 524, + 486, + 438, + 520, + 439, + 465, + 470, + 525, + 500, + 460, + 508, + 518, + 508, + 482, + 486, + 481, + 470, + 489, + 545, + 467, + 463, + 517, + 511, + 508, + 475, + 474, + 487, + 468, + 470, + 484, + 464, + 504, + 481, + 531, + 490, + 458, + 513, + 537, + 530, + 502, + 481, + 464, + 502, + 463, + 490, + 517, + 477, + 497, + 490, + 528, + 494, + 482, + 482, + 476, + 479, + 476, + 472, + 477, + 470, + 455, + 460, + 453, + 487, + 469, + 452, + 478, + 463, + 471, + 486, + 504, + 448, + 460, + 514, + 507, + 512, + 485, + 487, + 459, + 502, + 520, + 477, + 474, + 465, + 507, + 472, + 463, + 461, + 494, + 505, + 513, + 508, + 501, + 488, + 487, + 446, + 468, + 461, + 497, + 492, + 482, + 495, + 474, + 471, + 485, + 506, + 477, + 534, + 488, + 502, + 498, + 472, + 493, + 465, + 487, + 487, + 468, + 496, + 510, + 500, + 494, + 527, + 463, + 497, + 504, + 477, + 473, + 537, + 499, + 531, + 498, + 481, + 499, + 479, + 500, + 500, + 503, + 492, + 470, + 518, + 494, + 507, + 522, + 468, + 509, + 466, + 491, + 470, + 482, + 484, + 471, + 492, + 484, + 492, + 489, + 489, + 462, + 453, + 491, + 493, + 474, + 450, + 486, + 519, + 481, + 504, + 498, + 525, + 502, + 471, + 458, + 463, + 513, + 537, + 492, + 487, + 521, + 495, + 459, + 483, + 478, + 491, + 479, + 466, + 482, + 502, + 455, + 515, + 524, + 479, + 493, + 528, + 481, + 476, + 486, + 507, + 485, + 505, + 490, + 497, + 441, + 482, + 532, + 477, + 456, + 438, + 514, + 456, + 495, + 477, + 545, + 507, + 458, + 478, + 484, + 497, + 503, + 468, + 493, + 489, + 446, + 490, + 491, + 471, + 494, + 485, + 506, + 478, + 498, + 461, + 479, + 474, + 461, + 473, + 509, + 501, + 491, + 434, + 462, + 519, + 491, + 504, + 506, + 510, + 491, + 475, + 512, + 482, + 506, + 515, + 518, + 461, + 478, + 466, + 527, + 520, + 504, + 509, + 472, + 465, + 439, + 463, + 470, + 522, + 512, + 486, + 483, + 473, + 492, + 517, + 521, + 457, + 463, + 495, + 501, + 518, + 484, + 473, + 496, + 484, + 461, + 515, + 487, + 478, + 508, + 487, + 456, + 474, + 474, + 502, + 502, + 485, + 488, + 500, + 493, + 459, + 537, + 482, + 442, + 478, + 458, + 494, + 460, + 532, + 478, + 436, + 528, + 495, + 507, + 471, + 495, + 482, + 505, + 487, + 484, + 488, + 506, + 536, + 494, + 481, + 486, + 490, + 529, + 511, + 473, + 477, + 483, + 504, + 492, + 517, + 501, + 513, + 473, + 471, + 487, + 490, + 504, + 511, + 511, + 467, + 442, + 500, + 491, + 506, + 513, + 496, + 491, + 514, + 490, + 490, + 509, + 488, + 509, + 464, + 505, + 448, + 477, + 476, + 480, + 488, + 457, + 503, + 515, + 483, + 462, + 464, + 477, + 499, + 473, + 492, + 489, + 465, + 541, + 483, + 520, + 519, + 458, + 479, + 474, + 451, + 492, + 458, + 493, + 518, + 511, + 512, + 494, + 479, + 491, + 435, + 483, + 457, + 511, + 523, + 505, + 476, + 513, + 480, + 525, + 507, + 498, + 485, + 488, + 462, + 488, + 494, + 499, + 479, + 472, + 517, + 480, + 503, + 469, + 458, + 457, + 483, + 480, + 464, + 496, + 506, + 461, + 474, + 486, + 487, + 486, + 483, + 508, + 471, + 518, + 454, + 499, + 468, + 482, + 476, + 473, + 507, + 524, + 477, + 474, + 557, + 470, + 443, + 543, + 519, + 482, + 492, + 483, + 498, + 461, + 503, + 516, + 496, + 485, + 487, + 474, + 496, + 541, + 491, + 461, + 507, + 530, + 446, + 499, + 488, + 523, + 499, + 490, + 473, + 488, + 493, + 478, + 490, + 476, + 508, + 470, + 465, + 503, + 484, + 495, + 520, + 483, + 531, + 447, + 502, + 481, + 483, + 463, + 498, + 481, + 498, + 522, + 488, + 473, + 510, + 507, + 489, + 453, + 470, + 469, + 494, + 476, + 516, + 463, + 484, + 433, + 505, + 480, + 517, + 474, + 513, + 481, + 528, + 474, + 486, + 515, + 488, + 485, + 514, + 457, + 480, + 519, + 486, + 430, + 513, + 488, + 471, + 474, + 481, + 490, + 496, + 459, + 453, + 499, + 477, + 474, + 503, + 496, + 470, + 479, + 471, + 449, + 476, + 482, + 495, + 481, + 483, + 493, + 470, + 521, + 506, + 495, + 485, + 463, + 491, + 471, + 500, + 447, + 464, + 520, + 501, + 477, + 517, + 492, + 471, + 486, + 459, + 477, + 495, + 471, + 504, + 455, + 448, + 532, + 498, + 494, + 513, + 462, + 457, + 497, + 507, + 508, + 498, + 488, + 484, + 525, + 444, + 493, + 498, + 492, + 501, + 506, + 478 + ] + }, + { + "counters": [ + 495, + 433, + 458, + 465, + 476, + 491, + 493, + 463, + 501, + 466, + 487, + 490, + 482, + 435, + 458, + 487, + 483, + 500, + 521, + 516, + 476, + 527, + 486, + 452, + 482, + 513, + 478, + 500, + 491, + 473, + 496, + 464, + 489, + 498, + 479, + 526, + 511, + 547, + 489, + 504, + 457, + 522, + 501, + 513, + 463, + 516, + 466, + 462, + 500, + 505, + 533, + 473, + 444, + 508, + 504, + 495, + 488, + 504, + 488, + 502, + 509, + 447, + 473, + 494, + 516, + 481, + 465, + 497, + 475, + 517, + 451, + 477, + 470, + 483, + 446, + 486, + 517, + 495, + 481, + 505, + 487, + 487, + 493, + 527, + 474, + 500, + 458, + 497, + 513, + 521, + 468, + 491, + 496, + 502, + 468, + 492, + 436, + 483, + 491, + 478, + 505, + 516, + 489, + 508, + 500, + 438, + 491, + 471, + 454, + 475, + 507, + 507, + 503, + 449, + 463, + 465, + 462, + 483, + 467, + 482, + 473, + 471, + 526, + 461, + 461, + 488, + 460, + 468, + 449, + 483, + 485, + 471, + 476, + 490, + 508, + 486, + 489, + 492, + 526, + 481, + 452, + 493, + 463, + 472, + 480, + 492, + 537, + 466, + 537, + 478, + 489, + 468, + 512, + 480, + 512, + 479, + 473, + 477, + 472, + 471, + 468, + 477, + 475, + 486, + 490, + 506, + 500, + 490, + 508, + 496, + 455, + 479, + 507, + 449, + 521, + 474, + 507, + 484, + 531, + 485, + 488, + 508, + 506, + 450, + 464, + 481, + 493, + 487, + 460, + 450, + 487, + 493, + 452, + 459, + 526, + 492, + 497, + 484, + 487, + 527, + 500, + 517, + 485, + 504, + 485, + 484, + 469, + 530, + 469, + 453, + 493, + 485, + 472, + 531, + 475, + 490, + 480, + 504, + 495, + 508, + 496, + 528, + 458, + 484, + 468, + 536, + 495, + 473, + 455, + 513, + 477, + 485, + 459, + 499, + 479, + 461, + 477, + 517, + 502, + 490, + 460, + 499, + 475, + 484, + 505, + 502, + 470, + 482, + 530, + 503, + 478, + 481, + 488, + 490, + 493, + 490, + 481, + 492, + 436, + 485, + 505, + 489, + 513, + 518, + 531, + 474, + 479, + 476, + 495, + 460, + 516, + 473, + 470, + 505, + 467, + 472, + 567, + 474, + 482, + 513, + 457, + 502, + 522, + 476, + 483, + 471, + 506, + 458, + 467, + 472, + 460, + 444, + 533, + 509, + 516, + 505, + 495, + 462, + 499, + 507, + 493, + 453, + 458, + 474, + 475, + 476, + 508, + 494, + 469, + 477, + 462, + 468, + 510, + 502, + 495, + 421, + 499, + 459, + 542, + 481, + 481, + 503, + 487, + 458, + 465, + 510, + 482, + 466, + 468, + 468, + 485, + 486, + 478, + 448, + 493, + 524, + 540, + 529, + 470, + 494, + 481, + 478, + 500, + 495, + 487, + 529, + 484, + 508, + 467, + 531, + 477, + 509, + 466, + 473, + 481, + 497, + 500, + 524, + 468, + 505, + 542, + 457, + 493, + 463, + 495, + 515, + 492, + 521, + 496, + 486, + 466, + 478, + 499, + 489, + 464, + 475, + 458, + 526, + 505, + 498, + 499, + 457, + 491, + 497, + 500, + 478, + 494, + 521, + 513, + 492, + 533, + 460, + 507, + 493, + 486, + 502, + 513, + 481, + 512, + 523, + 487, + 498, + 475, + 472, + 536, + 509, + 491, + 449, + 500, + 477, + 516, + 474, + 484, + 488, + 490, + 469, + 493, + 495, + 473, + 454, + 472, + 478, + 497, + 491, + 527, + 491, + 493, + 473, + 475, + 488, + 466, + 468, + 486, + 502, + 454, + 476, + 476, + 495, + 476, + 485, + 445, + 464, + 505, + 501, + 486, + 525, + 538, + 478, + 487, + 495, + 506, + 487, + 484, + 496, + 525, + 491, + 493, + 502, + 452, + 506, + 504, + 491, + 517, + 472, + 466, + 516, + 466, + 486, + 464, + 517, + 516, + 480, + 508, + 493, + 499, + 518, + 515, + 507, + 469, + 517, + 494, + 475, + 457, + 471, + 482, + 490, + 502, + 483, + 492, + 499, + 486, + 487, + 514, + 478, + 507, + 513, + 500, + 512, + 487, + 460, + 519, + 523, + 483, + 469, + 478, + 462, + 492, + 474, + 431, + 478, + 482, + 479, + 510, + 486, + 514, + 480, + 533, + 495, + 446, + 486, + 506, + 482, + 498, + 475, + 487, + 541, + 478, + 464, + 528, + 505, + 485, + 530, + 476, + 489, + 497, + 491, + 493, + 501, + 449, + 522, + 483, + 482, + 463, + 507, + 477, + 443, + 457, + 467, + 477, + 480, + 501, + 486, + 516, + 484, + 460, + 508, + 501, + 493, + 494, + 497, + 460, + 456, + 520, + 492, + 496, + 479, + 458, + 480, + 449, + 501, + 482, + 457, + 483, + 498, + 505, + 535, + 479, + 462, + 535, + 503, + 457, + 510, + 502, + 510, + 492, + 503, + 493, + 492, + 453, + 479, + 486, + 436, + 489, + 494, + 504, + 517, + 484, + 475, + 477, + 436, + 491, + 478, + 477, + 493, + 529, + 482, + 436, + 513, + 513, + 491, + 526, + 481, + 456, + 490, + 469, + 501, + 482, + 480, + 494, + 520, + 502, + 460, + 463, + 488, + 496, + 501, + 462, + 493, + 486, + 523, + 482, + 482, + 478, + 475, + 473, + 507, + 460, + 491, + 461, + 524, + 498, + 479, + 477, + 513, + 492, + 478, + 475, + 510, + 517, + 468, + 488, + 492, + 473, + 468, + 482, + 512, + 483, + 512, + 473, + 468, + 509, + 497, + 480, + 494, + 482, + 551, + 499, + 471, + 472, + 473, + 477, + 508, + 492, + 508, + 441, + 497, + 498, + 481, + 488, + 526, + 483, + 502, + 507, + 484, + 516, + 506, + 473, + 479, + 505, + 538, + 456, + 455, + 479, + 506, + 465, + 456, + 510, + 447, + 498, + 493, + 459, + 503, + 492, + 496, + 524, + 494, + 474, + 482, + 503, + 507, + 496, + 482, + 484, + 499, + 499, + 513, + 442, + 514, + 478, + 475, + 483, + 487, + 484, + 497, + 537, + 478, + 515, + 496, + 494, + 461, + 490, + 494, + 464, + 481, + 465, + 455, + 494, + 506, + 469, + 490, + 470, + 492, + 475, + 491, + 450, + 469, + 458, + 496, + 489, + 491, + 481, + 492, + 471, + 452, + 512, + 480, + 496, + 500, + 516, + 494, + 504, + 496, + 485, + 524, + 461, + 499, + 478, + 516, + 497, + 481, + 488, + 499, + 474, + 498, + 487, + 467, + 483, + 505, + 468, + 498, + 474, + 475, + 509, + 468, + 502, + 471, + 505, + 469, + 477, + 493, + 476, + 477, + 441, + 470, + 479, + 487, + 452, + 540, + 530, + 535, + 527, + 512, + 469, + 520, + 488, + 472, + 461, + 468, + 514, + 505, + 494, + 511, + 492, + 517, + 541, + 488, + 477, + 509, + 454, + 503, + 495, + 518, + 521, + 476, + 506, + 488, + 464, + 479, + 457, + 515, + 470, + 473, + 504, + 521, + 489, + 469, + 460, + 529, + 524, + 499, + 470, + 480, + 494, + 486, + 509, + 510, + 508, + 439, + 475, + 497, + 510, + 474, + 500, + 481, + 477, + 515, + 464, + 498, + 523, + 491, + 492, + 485, + 512, + 481, + 492, + 437, + 474, + 486, + 519, + 470, + 483, + 475, + 489, + 521, + 482, + 482, + 532, + 494, + 534, + 502, + 487, + 496, + 487, + 511, + 507, + 502, + 499, + 506, + 477, + 463, + 493, + 525, + 534, + 484, + 500, + 511, + 518, + 488, + 510, + 428, + 471, + 482, + 461, + 500, + 509, + 462, + 479, + 512, + 471, + 461, + 468, + 469, + 501, + 494, + 470, + 486, + 507, + 511, + 433, + 520, + 535, + 497, + 473, + 450, + 535, + 492, + 471, + 473, + 507, + 429, + 427, + 510, + 458, + 413, + 470, + 478, + 490, + 470, + 496, + 468, + 492, + 483, + 464, + 488, + 532, + 462, + 476, + 516, + 459, + 488, + 497, + 505, + 497, + 457, + 474, + 448, + 489, + 502, + 468, + 509, + 506, + 521, + 466, + 484, + 521, + 476, + 492, + 476, + 481, + 490, + 468, + 497, + 477, + 462, + 467, + 497, + 475, + 521, + 490, + 470, + 471, + 507, + 486, + 481, + 511, + 501, + 499, + 508, + 460, + 478, + 491, + 487, + 520, + 497, + 478, + 472, + 458, + 465, + 496, + 482, + 493, + 492, + 486, + 491, + 527, + 474, + 495, + 511, + 471, + 486, + 485, + 494, + 495, + 478, + 465, + 507, + 479, + 508, + 507, + 500, + 482, + 516, + 498, + 492, + 521, + 524, + 503, + 488, + 493, + 463, + 518, + 537, + 483, + 480, + 465, + 527, + 495, + 493, + 508, + 482, + 495, + 521, + 492, + 494, + 481, + 475, + 443, + 490, + 479, + 477, + 494, + 471, + 484, + 431, + 452, + 436, + 501, + 480, + 477, + 467, + 490, + 484, + 474, + 483, + 508, + 507, + 507, + 530, + 525, + 489, + 497, + 502, + 507, + 485, + 478, + 516, + 472, + 473, + 485, + 477, + 465, + 513, + 463, + 487, + 514, + 489, + 454, + 476, + 480, + 484, + 479, + 513, + 462, + 486, + 520, + 504, + 515, + 458, + 513, + 526, + 491, + 451, + 489, + 520, + 515, + 454, + 463, + 481, + 493, + 498, + 508, + 482, + 474, + 489, + 457, + 461, + 481, + 499, + 482, + 524, + 509, + 511, + 447, + 509, + 482, + 473, + 478, + 507, + 471, + 486, + 532, + 516, + 474, + 489, + 540, + 510, + 514, + 477, + 497, + 463, + 492, + 507, + 515, + 495, + 510, + 469, + 487, + 482, + 472, + 504, + 506, + 503, + 522, + 501, + 491, + 482, + 487, + 530, + 474, + 498, + 478, + 477, + 491, + 495, + 460, + 512, + 474, + 460, + 483, + 518, + 532, + 487, + 483, + 509, + 529, + 490, + 515, + 483, + 487, + 448, + 488, + 494, + 474, + 454, + 495, + 473, + 479, + 472, + 462, + 502, + 439, + 464, + 519, + 456, + 501, + 436, + 487, + 490, + 502, + 468, + 442, + 483, + 504, + 473, + 495, + 528, + 475, + 498, + 483, + 529, + 476, + 489, + 509, + 474, + 457, + 470, + 487, + 480, + 451, + 475, + 459, + 485, + 490, + 493, + 492, + 421, + 501, + 480, + 446, + 482, + 486, + 501, + 497, + 512, + 482, + 456, + 502, + 451, + 489, + 515, + 462, + 494, + 475, + 496, + 502, + 495, + 471, + 490, + 440, + 491, + 489, + 506, + 487, + 490, + 459, + 491, + 459, + 469, + 468, + 445, + 487, + 494, + 491, + 490, + 474, + 454, + 480, + 503, + 476, + 491, + 459, + 444, + 480, + 497, + 423, + 512, + 508, + 528, + 460, + 501, + 495, + 518, + 477, + 506, + 498, + 496, + 507, + 468, + 462, + 462, + 508, + 502, + 468, + 485, + 472, + 475, + 487, + 493, + 522, + 484, + 513, + 513, + 525, + 491, + 513, + 479, + 414, + 494, + 473, + 450, + 494, + 474, + 497, + 525, + 465, + 507, + 505, + 456, + 492, + 458, + 465, + 457, + 534, + 537, + 475, + 481, + 481, + 470, + 500, + 507, + 487, + 469, + 463, + 473, + 509, + 497, + 456, + 485, + 490, + 494, + 486, + 501, + 481, + 529, + 492, + 517, + 488, + 501, + 493, + 486, + 481, + 547, + 509, + 505, + 507, + 480, + 510, + 491, + 524, + 519, + 469, + 488, + 493, + 464, + 467, + 519, + 441, + 493, + 505, + 496, + 481, + 470, + 528, + 467, + 495, + 499, + 490, + 487, + 480, + 537, + 495, + 471, + 469, + 497, + 487, + 494, + 447, + 454, + 453, + 493, + 523, + 473, + 453, + 510, + 499, + 465, + 508, + 509, + 499, + 446, + 493, + 511, + 520, + 484, + 503, + 458, + 447, + 454, + 459, + 474, + 461, + 483, + 507, + 455, + 464, + 498, + 485, + 492, + 472, + 456, + 468, + 525, + 508, + 459, + 493, + 472, + 501, + 503, + 485, + 471, + 510, + 439, + 494, + 494, + 500, + 490, + 468, + 497, + 516, + 467, + 481, + 473, + 474, + 501, + 512, + 487, + 517, + 461, + 495, + 475, + 508, + 480, + 497, + 515, + 530, + 489, + 490, + 454, + 483, + 475, + 492, + 521, + 488, + 495, + 492, + 480, + 524, + 484, + 479, + 457, + 481, + 526, + 502, + 480, + 476, + 452, + 530, + 481, + 518, + 498, + 509, + 439, + 472, + 492, + 490, + 517, + 508, + 502, + 508, + 511, + 488, + 510, + 504, + 491, + 509, + 462, + 457, + 474, + 508, + 523, + 488, + 481, + 539, + 492, + 454, + 523, + 511, + 500, + 493, + 436, + 461, + 450, + 515, + 497, + 462, + 490, + 483, + 419, + 510, + 475, + 461, + 468, + 506, + 492, + 464, + 469, + 501, + 481, + 512, + 473, + 513, + 462, + 500, + 513, + 475, + 462, + 484, + 491, + 470, + 452, + 459, + 516, + 529, + 539, + 479, + 486, + 489, + 508, + 515, + 502, + 465, + 471, + 486, + 479, + 464, + 492, + 481, + 490, + 478, + 458, + 514, + 502, + 506, + 477, + 478, + 505, + 466, + 531, + 474, + 475, + 498, + 517, + 504, + 480, + 537, + 481, + 480, + 471, + 487, + 480, + 466, + 484, + 499, + 465, + 466, + 503, + 420, + 496, + 525, + 512, + 520, + 451, + 495, + 502, + 482, + 484, + 492, + 507, + 495, + 479, + 505, + 489, + 513, + 466, + 464, + 482, + 511, + 467, + 483, + 514, + 483, + 490, + 522, + 478, + 455, + 473, + 511, + 485, + 479, + 499, + 499, + 487, + 499, + 444, + 493, + 491, + 477, + 511, + 456, + 507, + 529, + 506, + 476, + 441, + 508, + 495, + 477, + 494, + 461, + 516, + 525, + 490, + 488, + 496, + 515, + 479, + 582, + 513, + 463, + 496, + 498, + 442, + 477, + 509, + 499, + 473, + 495, + 490, + 525, + 474, + 496, + 458, + 501, + 471, + 474, + 501, + 477, + 503, + 465, + 517, + 520, + 469, + 506, + 492, + 508, + 481, + 488, + 507, + 491, + 531, + 526, + 463, + 517, + 452, + 474, + 488, + 503, + 492, + 505, + 442, + 474, + 461, + 493, + 466, + 493, + 461, + 486, + 475, + 480, + 507, + 502, + 493, + 490, + 490, + 483, + 479, + 484, + 507, + 451, + 465, + 450, + 517, + 502, + 502, + 485, + 481, + 471, + 498, + 465, + 462, + 484, + 437, + 460, + 495, + 438, + 462, + 462, + 510, + 529, + 484, + 485, + 488, + 509, + 453, + 499, + 500, + 499, + 479, + 528, + 487, + 483, + 481, + 474, + 489, + 478, + 475, + 471, + 486, + 476, + 479, + 534, + 480, + 494, + 482, + 507, + 530, + 487, + 497, + 473, + 473, + 472, + 458, + 442, + 440, + 459, + 505, + 502, + 498, + 467, + 500, + 505, + 517, + 522, + 479, + 494, + 487, + 457, + 472, + 480, + 517, + 483, + 514, + 486, + 471, + 457, + 504, + 471, + 466, + 494, + 508, + 465, + 487, + 522, + 500, + 484, + 487, + 482, + 450, + 494, + 527, + 483, + 479, + 472, + 521, + 464, + 486, + 512, + 489, + 503, + 478, + 492, + 527, + 507, + 528, + 507, + 478, + 462, + 461, + 462, + 497, + 543, + 498, + 477, + 492, + 493, + 466, + 480, + 509, + 530, + 448, + 522, + 475, + 484, + 455, + 444, + 477, + 491, + 494, + 450, + 494, + 507, + 515, + 447, + 512, + 506, + 486, + 452, + 491, + 472, + 513, + 511, + 503, + 504, + 477, + 501, + 516, + 494, + 480, + 516, + 482, + 480, + 483, + 440, + 492, + 464, + 501, + 498, + 506, + 500, + 497, + 472, + 527, + 520, + 459, + 504, + 495, + 474, + 489, + 452, + 499, + 485, + 466, + 512, + 500, + 485, + 516, + 495, + 476, + 514, + 494, + 497, + 492, + 476, + 501, + 501, + 465, + 510, + 502, + 476, + 525, + 517, + 526, + 521, + 491, + 481, + 480, + 455, + 469, + 477, + 523, + 515, + 511, + 429, + 481, + 522, + 515, + 473, + 481, + 511, + 492, + 479, + 478, + 479, + 526, + 499, + 438, + 491, + 535, + 497, + 426, + 495, + 489, + 465, + 467, + 485, + 443, + 461, + 466, + 462, + 467, + 553, + 502, + 504, + 496, + 481, + 454, + 454, + 458, + 528, + 493, + 482, + 505, + 495, + 522, + 473, + 494, + 442, + 499, + 484, + 496, + 561, + 438, + 512, + 468, + 514, + 472, + 464, + 503, + 526, + 480, + 512, + 467, + 465, + 444, + 504, + 503, + 486, + 490, + 492, + 490, + 524, + 474, + 491, + 488, + 497, + 514, + 483, + 452, + 472, + 532, + 491, + 481, + 487, + 509, + 512, + 507, + 490, + 456, + 511, + 507, + 477, + 452, + 510, + 469, + 481, + 467, + 497, + 461, + 510, + 487, + 491, + 508, + 467, + 463, + 433, + 500, + 488, + 483, + 460, + 508, + 512, + 447, + 470, + 466, + 492, + 502, + 510, + 481, + 537, + 515, + 524, + 506, + 501, + 535, + 526, + 537, + 492, + 466, + 518, + 502, + 528, + 494, + 416, + 483, + 465, + 505, + 472, + 518 + ] + } + ], + "default_value": 0 + }, + "null_count": 0, + "tot_col_size": 2991746, + "last_update_version": 417600050985172992, + "correlation": 1 + }, + "b": { + "histogram": { + "ndv": 1000000, + "buckets": [ + { + "count": 3998, + "lower_bound": "ODQz", + "upper_bound": "MjkyMQ==", + "repeats": 1 + }, + { + "count": 7996, + "lower_bound": "NDAwOQ==", + "upper_bound": "NzQyOQ==", + "repeats": 1 + }, + { + "count": 11994, + "lower_bound": "NzUxOA==", + "upper_bound": "MTA0NTE=", + "repeats": 1 + }, + { + "count": 15992, + "lower_bound": "MTA5OTA=", + "upper_bound": "MTI2MTQ=", + "repeats": 1 + }, + { + "count": 19990, + "lower_bound": "MTM0MTY=", + "upper_bound": "MTYwNTA=", + "repeats": 1 + }, + { + "count": 23988, + "lower_bound": "MTY0OTE=", + "upper_bound": "MjA3MzY=", + "repeats": 1 + }, + { + "count": 27986, + "lower_bound": "MjA5ODY=", + "upper_bound": "MjM1MTA=", + "repeats": 1 + }, + { + "count": 31984, + "lower_bound": "MjQwNDI=", + "upper_bound": "Mjc4ODg=", + "repeats": 1 + }, + { + "count": 35982, + "lower_bound": "MjgwMjY=", + "upper_bound": "MzA0NTc=", + "repeats": 1 + }, + { + "count": 39980, + "lower_bound": "MzA3Mjg=", + "upper_bound": "MzQxNjU=", + "repeats": 1 + }, + { + "count": 43978, + "lower_bound": "MzQzMzI=", + "upper_bound": "MzY2NjI=", + "repeats": 1 + }, + { + "count": 47976, + "lower_bound": "MzcwODk=", + "upper_bound": "NDEyOTc=", + "repeats": 1 + }, + { + "count": 51974, + "lower_bound": "NDEzMzk=", + "upper_bound": "NDQxNzI=", + "repeats": 1 + }, + { + "count": 55972, + "lower_bound": "NDQzNzc=", + "upper_bound": "NDY1Nzc=", + "repeats": 1 + }, + { + "count": 59970, + "lower_bound": "NDc0NDM=", + "upper_bound": "NDg3NzI=", + "repeats": 1 + }, + { + "count": 63968, + "lower_bound": "NDkxNDg=", + "upper_bound": "NTI5OTE=", + "repeats": 1 + }, + { + "count": 67966, + "lower_bound": "NTM1NzQ=", + "upper_bound": "NTU5NjE=", + "repeats": 1 + }, + { + "count": 71964, + "lower_bound": "NTU5NjM=", + "upper_bound": "NTc2NzQ=", + "repeats": 1 + }, + { + "count": 75962, + "lower_bound": "NTc5Njg=", + "upper_bound": "NjEyODU=", + "repeats": 1 + }, + { + "count": 79960, + "lower_bound": "NjEzMDU=", + "upper_bound": "NjM4MTQ=", + "repeats": 1 + }, + { + "count": 83958, + "lower_bound": "NjM5MDY=", + "upper_bound": "NjYyNDc=", + "repeats": 1 + }, + { + "count": 87956, + "lower_bound": "NjY5MDY=", + "upper_bound": "NzA0Njk=", + "repeats": 1 + }, + { + "count": 91954, + "lower_bound": "NzA0ODQ=", + "upper_bound": "NzI2MTE=", + "repeats": 1 + }, + { + "count": 95952, + "lower_bound": "NzI4NzQ=", + "upper_bound": "NzU1MzE=", + "repeats": 1 + }, + { + "count": 99950, + "lower_bound": "NzU2NTc=", + "upper_bound": "NzY5MzY=", + "repeats": 1 + }, + { + "count": 103948, + "lower_bound": "NzcxNTU=", + "upper_bound": "NzkyMTc=", + "repeats": 1 + }, + { + "count": 107946, + "lower_bound": "NzkyOTc=", + "upper_bound": "ODIxMzQ=", + "repeats": 1 + }, + { + "count": 111944, + "lower_bound": "ODI2NDY=", + "upper_bound": "ODQwMTQ=", + "repeats": 1 + }, + { + "count": 115942, + "lower_bound": "ODQyNjA=", + "upper_bound": "ODYwNzQ=", + "repeats": 1 + }, + { + "count": 119940, + "lower_bound": "ODYyMTI=", + "upper_bound": "ODg2MjY=", + "repeats": 1 + }, + { + "count": 123938, + "lower_bound": "ODg4Njc=", + "upper_bound": "OTE3Mzg=", + "repeats": 1 + }, + { + "count": 127936, + "lower_bound": "OTI0NDA=", + "upper_bound": "OTUzNzY=", + "repeats": 1 + }, + { + "count": 131934, + "lower_bound": "OTU2MTA=", + "upper_bound": "OTk1OTU=", + "repeats": 1 + }, + { + "count": 135932, + "lower_bound": "OTk4Njc=", + "upper_bound": "MTAzMDE4", + "repeats": 1 + }, + { + "count": 139930, + "lower_bound": "MTAzMDQ5", + "upper_bound": "MTA0NDU0", + "repeats": 1 + }, + { + "count": 143928, + "lower_bound": "MTA0NTY5", + "upper_bound": "MTA5NTg0", + "repeats": 1 + }, + { + "count": 147926, + "lower_bound": "MTEwMDI0", + "upper_bound": "MTEzMzEy", + "repeats": 1 + }, + { + "count": 151924, + "lower_bound": "MTEzNDYx", + "upper_bound": "MTE1OTkz", + "repeats": 1 + }, + { + "count": 155922, + "lower_bound": "MTE2NzU0", + "upper_bound": "MTIwOTcx", + "repeats": 1 + }, + { + "count": 159920, + "lower_bound": "MTIxNDU3", + "upper_bound": "MTIzMzY1", + "repeats": 1 + }, + { + "count": 163918, + "lower_bound": "MTI0MTY4", + "upper_bound": "MTI2MTk0", + "repeats": 1 + }, + { + "count": 167916, + "lower_bound": "MTI2Mzc2", + "upper_bound": "MTI5OTk1", + "repeats": 1 + }, + { + "count": 171914, + "lower_bound": "MTMwMTg0", + "upper_bound": "MTM2MDAx", + "repeats": 1 + }, + { + "count": 175912, + "lower_bound": "MTM2MDM0", + "upper_bound": "MTM4ODY3", + "repeats": 1 + }, + { + "count": 179910, + "lower_bound": "MTM4OTM4", + "upper_bound": "MTQyNTMw", + "repeats": 1 + }, + { + "count": 183908, + "lower_bound": "MTQzNDI3", + "upper_bound": "MTQ1NDc0", + "repeats": 1 + }, + { + "count": 187906, + "lower_bound": "MTQ1OTcz", + "upper_bound": "MTQ4MDM4", + "repeats": 1 + }, + { + "count": 191904, + "lower_bound": "MTQ4MTAx", + "upper_bound": "MTUxMzM1", + "repeats": 1 + }, + { + "count": 195902, + "lower_bound": "MTUyMzY5", + "upper_bound": "MTU1NDc3", + "repeats": 1 + }, + { + "count": 199900, + "lower_bound": "MTU1NTc0", + "upper_bound": "MTU3MzM3", + "repeats": 1 + }, + { + "count": 203898, + "lower_bound": "MTU3NTEz", + "upper_bound": "MTU5NDE2", + "repeats": 1 + }, + { + "count": 207896, + "lower_bound": "MTU5NTE1", + "upper_bound": "MTY0OTg1", + "repeats": 1 + }, + { + "count": 211894, + "lower_bound": "MTY1Mjg4", + "upper_bound": "MTY3Mjcy", + "repeats": 1 + }, + { + "count": 215892, + "lower_bound": "MTY3Njcz", + "upper_bound": "MTcxMzU0", + "repeats": 1 + }, + { + "count": 219890, + "lower_bound": "MTcxOTAy", + "upper_bound": "MTc0MDkw", + "repeats": 1 + }, + { + "count": 223888, + "lower_bound": "MTc0MzUy", + "upper_bound": "MTc3Mzc4", + "repeats": 1 + }, + { + "count": 227886, + "lower_bound": "MTc3NTMw", + "upper_bound": "MTgwOTEy", + "repeats": 1 + }, + { + "count": 231884, + "lower_bound": "MTgxMzIx", + "upper_bound": "MTgzMDM1", + "repeats": 1 + }, + { + "count": 235882, + "lower_bound": "MTgzNDYw", + "upper_bound": "MTg0OTcw", + "repeats": 1 + }, + { + "count": 239880, + "lower_bound": "MTg1NTQ0", + "upper_bound": "MTg4NDg0", + "repeats": 1 + }, + { + "count": 243878, + "lower_bound": "MTg4NzU1", + "upper_bound": "MTkyODQ2", + "repeats": 1 + }, + { + "count": 247876, + "lower_bound": "MTkzMTk2", + "upper_bound": "MTk1NjA5", + "repeats": 1 + }, + { + "count": 251874, + "lower_bound": "MTk1NzQ1", + "upper_bound": "MTk3NjAy", + "repeats": 1 + }, + { + "count": 255872, + "lower_bound": "MTk3NzIy", + "upper_bound": "MjAxOTAw", + "repeats": 1 + }, + { + "count": 259870, + "lower_bound": "MjAyMDMy", + "upper_bound": "MjAzNjkz", + "repeats": 1 + }, + { + "count": 263868, + "lower_bound": "MjAzOTc0", + "upper_bound": "MjA1OTE1", + "repeats": 1 + }, + { + "count": 267866, + "lower_bound": "MjA2MzEz", + "upper_bound": "MjA4MDc3", + "repeats": 1 + }, + { + "count": 271864, + "lower_bound": "MjA4MTgw", + "upper_bound": "MjA5ODMx", + "repeats": 1 + }, + { + "count": 275862, + "lower_bound": "MjEwMjkw", + "upper_bound": "MjEyMTEw", + "repeats": 1 + }, + { + "count": 279860, + "lower_bound": "MjEyNDg1", + "upper_bound": "MjE1NTY5", + "repeats": 1 + }, + { + "count": 283858, + "lower_bound": "MjE1NTg0", + "upper_bound": "MjE4MzQz", + "repeats": 1 + }, + { + "count": 287856, + "lower_bound": "MjE4OTI3", + "upper_bound": "MjIwNTk4", + "repeats": 1 + }, + { + "count": 291854, + "lower_bound": "MjIwNjQ4", + "upper_bound": "MjIyNzM1", + "repeats": 1 + }, + { + "count": 295852, + "lower_bound": "MjIyODgx", + "upper_bound": "MjI0NDM4", + "repeats": 1 + }, + { + "count": 299850, + "lower_bound": "MjI0ODUz", + "upper_bound": "MjI4MjIz", + "repeats": 1 + }, + { + "count": 303848, + "lower_bound": "MjI4Mjkz", + "upper_bound": "MjMxNDMz", + "repeats": 1 + }, + { + "count": 307846, + "lower_bound": "MjMyNDgw", + "upper_bound": "MjMzNjQ0", + "repeats": 1 + }, + { + "count": 311844, + "lower_bound": "MjMzNzYw", + "upper_bound": "MjM1NjI4", + "repeats": 1 + }, + { + "count": 315842, + "lower_bound": "MjM1NjUy", + "upper_bound": "MjM5MTEw", + "repeats": 1 + }, + { + "count": 319840, + "lower_bound": "MjM5MzUy", + "upper_bound": "MjQxNDk2", + "repeats": 1 + }, + { + "count": 323838, + "lower_bound": "MjQxNzA0", + "upper_bound": "MjQzMTQ2", + "repeats": 1 + }, + { + "count": 327836, + "lower_bound": "MjQzMjM5", + "upper_bound": "MjQ1ODE0", + "repeats": 1 + }, + { + "count": 331834, + "lower_bound": "MjQ1ODYw", + "upper_bound": "MjQ3NjA1", + "repeats": 1 + }, + { + "count": 335832, + "lower_bound": "MjQ3NjU2", + "upper_bound": "MjQ5OTYw", + "repeats": 1 + }, + { + "count": 339830, + "lower_bound": "MjUwMzcz", + "upper_bound": "MjUzMzk3", + "repeats": 1 + }, + { + "count": 343828, + "lower_bound": "MjUzNDgy", + "upper_bound": "MjU1MjEx", + "repeats": 1 + }, + { + "count": 347826, + "lower_bound": "MjU1NzMy", + "upper_bound": "MjU5OTY4", + "repeats": 1 + }, + { + "count": 351824, + "lower_bound": "MjYwMTcw", + "upper_bound": "MjYyNzky", + "repeats": 1 + }, + { + "count": 355822, + "lower_bound": "MjYzMTU1", + "upper_bound": "MjY1MDkz", + "repeats": 1 + }, + { + "count": 359820, + "lower_bound": "MjY1NTU3", + "upper_bound": "MjY3MzA4", + "repeats": 1 + }, + { + "count": 363818, + "lower_bound": "MjY3MzMw", + "upper_bound": "MjcxMDI2", + "repeats": 1 + }, + { + "count": 367816, + "lower_bound": "MjcxODcz", + "upper_bound": "Mjc0NDM1", + "repeats": 1 + }, + { + "count": 371814, + "lower_bound": "Mjc0NTE2", + "upper_bound": "Mjc3MDQ1", + "repeats": 1 + }, + { + "count": 375812, + "lower_bound": "Mjc3Njc2", + "upper_bound": "MjgwOTk2", + "repeats": 1 + }, + { + "count": 379810, + "lower_bound": "MjgxMjE0", + "upper_bound": "Mjg0NzI0", + "repeats": 1 + }, + { + "count": 383808, + "lower_bound": "Mjg2NTU2", + "upper_bound": "Mjg3Nzg1", + "repeats": 1 + }, + { + "count": 387806, + "lower_bound": "Mjg4MTQy", + "upper_bound": "MjkwMjA2", + "repeats": 1 + }, + { + "count": 391804, + "lower_bound": "MjkwMjg1", + "upper_bound": "MjkyNTI4", + "repeats": 1 + }, + { + "count": 395802, + "lower_bound": "MjkyNTcx", + "upper_bound": "Mjk1MjA3", + "repeats": 1 + }, + { + "count": 399800, + "lower_bound": "Mjk1NDA2", + "upper_bound": "Mjk4MTMx", + "repeats": 1 + }, + { + "count": 403798, + "lower_bound": "Mjk4MTgx", + "upper_bound": "MzAwOTY2", + "repeats": 1 + }, + { + "count": 407796, + "lower_bound": "MzAwOTk1", + "upper_bound": "MzAzNzc3", + "repeats": 1 + }, + { + "count": 411794, + "lower_bound": "MzA0MDQ4", + "upper_bound": "MzA3OTg4", + "repeats": 1 + }, + { + "count": 415792, + "lower_bound": "MzA4MDM0", + "upper_bound": "MzEwMzcy", + "repeats": 1 + }, + { + "count": 419790, + "lower_bound": "MzEwNDM1", + "upper_bound": "MzE0NjQy", + "repeats": 1 + }, + { + "count": 423788, + "lower_bound": "MzE1Njg2", + "upper_bound": "MzE3NTk4", + "repeats": 1 + }, + { + "count": 427786, + "lower_bound": "MzE4Mzc5", + "upper_bound": "MzE5NDcx", + "repeats": 1 + }, + { + "count": 431784, + "lower_bound": "MzE5OTU2", + "upper_bound": "MzIxNDk1", + "repeats": 1 + }, + { + "count": 435782, + "lower_bound": "MzIxNTc3", + "upper_bound": "MzIzMDk1", + "repeats": 1 + }, + { + "count": 439780, + "lower_bound": "MzIzMDk4", + "upper_bound": "MzI0NjM0", + "repeats": 1 + }, + { + "count": 443778, + "lower_bound": "MzI0NzMx", + "upper_bound": "MzI3MDQx", + "repeats": 1 + }, + { + "count": 447776, + "lower_bound": "MzI3NTg5", + "upper_bound": "MzI5OTcy", + "repeats": 1 + }, + { + "count": 451774, + "lower_bound": "MzMwMTEy", + "upper_bound": "MzMzMjM0", + "repeats": 1 + }, + { + "count": 455772, + "lower_bound": "MzMzMzc2", + "upper_bound": "MzM1NjYz", + "repeats": 1 + }, + { + "count": 459770, + "lower_bound": "MzM2NDMy", + "upper_bound": "MzM4NzA4", + "repeats": 1 + }, + { + "count": 463768, + "lower_bound": "MzM4ODYz", + "upper_bound": "MzQwNjc5", + "repeats": 1 + }, + { + "count": 467766, + "lower_bound": "MzQwNzIw", + "upper_bound": "MzQzNDE0", + "repeats": 1 + }, + { + "count": 471764, + "lower_bound": "MzQzNzE2", + "upper_bound": "MzQ1NDEy", + "repeats": 1 + }, + { + "count": 475762, + "lower_bound": "MzQ1NTU4", + "upper_bound": "MzQ4NTA5", + "repeats": 1 + }, + { + "count": 479760, + "lower_bound": "MzQ5MzM3", + "upper_bound": "MzUwNjc5", + "repeats": 1 + }, + { + "count": 483758, + "lower_bound": "MzUwODAx", + "upper_bound": "MzUzMjU2", + "repeats": 1 + }, + { + "count": 487756, + "lower_bound": "MzUzMzMx", + "upper_bound": "MzU2MDk1", + "repeats": 1 + }, + { + "count": 491754, + "lower_bound": "MzU2NDcz", + "upper_bound": "MzU4MTUy", + "repeats": 1 + }, + { + "count": 495752, + "lower_bound": "MzU4NDAx", + "upper_bound": "MzYyMDcz", + "repeats": 1 + }, + { + "count": 499750, + "lower_bound": "MzYyNDky", + "upper_bound": "MzY1ODkx", + "repeats": 1 + }, + { + "count": 503748, + "lower_bound": "MzY1OTI0", + "upper_bound": "MzY4NTc3", + "repeats": 1 + }, + { + "count": 507746, + "lower_bound": "MzY5NDEy", + "upper_bound": "MzczNjQ0", + "repeats": 1 + }, + { + "count": 511744, + "lower_bound": "Mzc0MjIx", + "upper_bound": "Mzc3MjU3", + "repeats": 1 + }, + { + "count": 515742, + "lower_bound": "Mzc4NjAw", + "upper_bound": "Mzg5Njg1", + "repeats": 1 + }, + { + "count": 519740, + "lower_bound": "Mzg5ODI1", + "upper_bound": "Mzk0NTcx", + "repeats": 1 + }, + { + "count": 523738, + "lower_bound": "Mzk1ODc2", + "upper_bound": "NDAyNTg1", + "repeats": 1 + }, + { + "count": 527736, + "lower_bound": "NDAzODYy", + "upper_bound": "NDA4ODI1", + "repeats": 1 + }, + { + "count": 531734, + "lower_bound": "NDA5MTI4", + "upper_bound": "NDEzNDc5", + "repeats": 1 + }, + { + "count": 535732, + "lower_bound": "NDE0MTQ5", + "upper_bound": "NDE5MDc2", + "repeats": 1 + }, + { + "count": 539730, + "lower_bound": "NDE5MzUy", + "upper_bound": "NDI3MjE4", + "repeats": 1 + }, + { + "count": 543728, + "lower_bound": "NDI3MzU2", + "upper_bound": "NDMxNjA5", + "repeats": 1 + }, + { + "count": 547726, + "lower_bound": "NDMxNjM2", + "upper_bound": "NDM0MDk1", + "repeats": 1 + }, + { + "count": 551724, + "lower_bound": "NDM0Njk2", + "upper_bound": "NDM3Nzg0", + "repeats": 1 + }, + { + "count": 555722, + "lower_bound": "NDM4MjA3", + "upper_bound": "NDQ0MDk0", + "repeats": 1 + }, + { + "count": 559720, + "lower_bound": "NDQ0MTYz", + "upper_bound": "NDUxNjEy", + "repeats": 1 + }, + { + "count": 563718, + "lower_bound": "NDUxOTM5", + "upper_bound": "NDU1MjUw", + "repeats": 1 + }, + { + "count": 567716, + "lower_bound": "NDU1Nzgx", + "upper_bound": "NDY0MjQ1", + "repeats": 1 + }, + { + "count": 571714, + "lower_bound": "NDY1MDQz", + "upper_bound": "NDY4MDY1", + "repeats": 1 + }, + { + "count": 575712, + "lower_bound": "NDY5OTI4", + "upper_bound": "NDc4MDE2", + "repeats": 1 + }, + { + "count": 579710, + "lower_bound": "NDc4MDYx", + "upper_bound": "NDgzNTU4", + "repeats": 1 + }, + { + "count": 583708, + "lower_bound": "NDgzNTYx", + "upper_bound": "NDg3MjM0", + "repeats": 1 + }, + { + "count": 587706, + "lower_bound": "NDg3Mjc0", + "upper_bound": "NDkzMTc1", + "repeats": 1 + }, + { + "count": 591704, + "lower_bound": "NDkzNjc2", + "upper_bound": "NDk3MTk5", + "repeats": 1 + }, + { + "count": 595702, + "lower_bound": "NDk3MjM1", + "upper_bound": "NTAzMjMz", + "repeats": 1 + }, + { + "count": 599700, + "lower_bound": "NTAzOTU5", + "upper_bound": "NTA3MzAx", + "repeats": 1 + }, + { + "count": 603698, + "lower_bound": "NTA3NDY5", + "upper_bound": "NTEwMjMx", + "repeats": 1 + }, + { + "count": 607696, + "lower_bound": "NTExNzI0", + "upper_bound": "NTE3OTY1", + "repeats": 1 + }, + { + "count": 611694, + "lower_bound": "NTE4NTg5", + "upper_bound": "NTIxNzIy", + "repeats": 1 + }, + { + "count": 615692, + "lower_bound": "NTIxNzU2", + "upper_bound": "NTI2Mjgx", + "repeats": 1 + }, + { + "count": 619690, + "lower_bound": "NTI3OTU2", + "upper_bound": "NTMxNjMy", + "repeats": 1 + }, + { + "count": 623688, + "lower_bound": "NTMxNzgz", + "upper_bound": "NTM0Nzg0", + "repeats": 1 + }, + { + "count": 627686, + "lower_bound": "NTM1NTI4", + "upper_bound": "NTM4NTg3", + "repeats": 1 + }, + { + "count": 631684, + "lower_bound": "NTQwODQw", + "upper_bound": "NTQ0MzY5", + "repeats": 1 + }, + { + "count": 635682, + "lower_bound": "NTQ2MDc3", + "upper_bound": "NTUwNDg0", + "repeats": 1 + }, + { + "count": 639680, + "lower_bound": "NTUxMzgx", + "upper_bound": "NTU2MDM2", + "repeats": 1 + }, + { + "count": 643678, + "lower_bound": "NTU2OTkx", + "upper_bound": "NTU5MTg0", + "repeats": 1 + }, + { + "count": 647676, + "lower_bound": "NTU5NDc3", + "upper_bound": "NTYyODQw", + "repeats": 1 + }, + { + "count": 651674, + "lower_bound": "NTYzMzEz", + "upper_bound": "NTY1ODE0", + "repeats": 1 + }, + { + "count": 655672, + "lower_bound": "NTY4NzQ2", + "upper_bound": "NTczNjQ1", + "repeats": 1 + }, + { + "count": 659670, + "lower_bound": "NTczODMx", + "upper_bound": "NTc3MjA0", + "repeats": 1 + }, + { + "count": 663668, + "lower_bound": "NTc5NDYx", + "upper_bound": "NTg0MTQz", + "repeats": 1 + }, + { + "count": 667666, + "lower_bound": "NTg0NzI4", + "upper_bound": "NTg5NjI3", + "repeats": 1 + }, + { + "count": 671664, + "lower_bound": "NTg5NjQ2", + "upper_bound": "NTk1MjM3", + "repeats": 1 + }, + { + "count": 675662, + "lower_bound": "NTk1MzU4", + "upper_bound": "NjAwOTc2", + "repeats": 1 + }, + { + "count": 679660, + "lower_bound": "NjAyMDcz", + "upper_bound": "NjA0ODU1", + "repeats": 1 + }, + { + "count": 683658, + "lower_bound": "NjA1NDkx", + "upper_bound": "NjEwMTQz", + "repeats": 1 + }, + { + "count": 687656, + "lower_bound": "NjEwMTk2", + "upper_bound": "NjE0MTM5", + "repeats": 1 + }, + { + "count": 691654, + "lower_bound": "NjE0MzM3", + "upper_bound": "NjIxNzgz", + "repeats": 1 + }, + { + "count": 695652, + "lower_bound": "NjIzMTEy", + "upper_bound": "NjI1NDYz", + "repeats": 1 + }, + { + "count": 699650, + "lower_bound": "NjI1Nzkw", + "upper_bound": "NjI4NTQ2", + "repeats": 1 + }, + { + "count": 703648, + "lower_bound": "NjI4OTky", + "upper_bound": "NjM1ODk4", + "repeats": 1 + }, + { + "count": 707646, + "lower_bound": "NjM3NTk2", + "upper_bound": "NjQzMjgy", + "repeats": 1 + }, + { + "count": 711644, + "lower_bound": "NjQzMzc2", + "upper_bound": "NjQ3MDM3", + "repeats": 1 + }, + { + "count": 715642, + "lower_bound": "NjQ3NzY1", + "upper_bound": "NjUwOTEx", + "repeats": 1 + }, + { + "count": 719640, + "lower_bound": "NjUxNzM1", + "upper_bound": "NjU0NTE5", + "repeats": 1 + }, + { + "count": 723638, + "lower_bound": "NjU0OTA2", + "upper_bound": "NjU2Njg0", + "repeats": 1 + }, + { + "count": 727636, + "lower_bound": "NjU3MTUx", + "upper_bound": "NjYyMTQ5", + "repeats": 1 + }, + { + "count": 731634, + "lower_bound": "NjYyNjQw", + "upper_bound": "NjY1MjM3", + "repeats": 1 + }, + { + "count": 735632, + "lower_bound": "NjY2MDEx", + "upper_bound": "NjcyOTcx", + "repeats": 1 + }, + { + "count": 739630, + "lower_bound": "Njc0Nzcy", + "upper_bound": "NjgyMzc4", + "repeats": 1 + }, + { + "count": 743628, + "lower_bound": "NjgyNzY4", + "upper_bound": "Njg4MTU0", + "repeats": 1 + }, + { + "count": 747626, + "lower_bound": "Njg4MzMx", + "upper_bound": "NjkyNTE1", + "repeats": 1 + }, + { + "count": 751624, + "lower_bound": "NjkyNTQz", + "upper_bound": "Njk3ODA1", + "repeats": 1 + }, + { + "count": 755622, + "lower_bound": "Njk4NjQw", + "upper_bound": "NzAxNzQ0", + "repeats": 1 + }, + { + "count": 759620, + "lower_bound": "NzAyMDg3", + "upper_bound": "NzA0MTU1", + "repeats": 1 + }, + { + "count": 763618, + "lower_bound": "NzA0NTcx", + "upper_bound": "NzA4MjY1", + "repeats": 1 + }, + { + "count": 767616, + "lower_bound": "NzA4MzAz", + "upper_bound": "NzEyNzUz", + "repeats": 1 + }, + { + "count": 771614, + "lower_bound": "NzE0NDg1", + "upper_bound": "NzE4ODc5", + "repeats": 1 + }, + { + "count": 775612, + "lower_bound": "NzIwMDky", + "upper_bound": "NzIzMTk4", + "repeats": 1 + }, + { + "count": 779610, + "lower_bound": "NzIzOTYy", + "upper_bound": "NzI3Njg3", + "repeats": 1 + }, + { + "count": 783608, + "lower_bound": "NzI4MDQ3", + "upper_bound": "NzMyOTAw", + "repeats": 1 + }, + { + "count": 787606, + "lower_bound": "NzM0NTYx", + "upper_bound": "NzM5MTU0", + "repeats": 1 + }, + { + "count": 791604, + "lower_bound": "NzM5OTk5", + "upper_bound": "NzQ1NDg4", + "repeats": 1 + }, + { + "count": 795602, + "lower_bound": "NzQ1NDk3", + "upper_bound": "NzUwNzU0", + "repeats": 1 + }, + { + "count": 799600, + "lower_bound": "NzUwODk3", + "upper_bound": "NzU1NDA0", + "repeats": 1 + }, + { + "count": 803598, + "lower_bound": "NzU3NTI5", + "upper_bound": "NzYxMDgy", + "repeats": 1 + }, + { + "count": 807596, + "lower_bound": "NzYxMDkw", + "upper_bound": "NzY2ODM2", + "repeats": 1 + }, + { + "count": 811594, + "lower_bound": "NzY4MTMy", + "upper_bound": "NzcwNjE4", + "repeats": 1 + }, + { + "count": 815592, + "lower_bound": "NzcxODUx", + "upper_bound": "Nzc1MDI2", + "repeats": 1 + }, + { + "count": 819590, + "lower_bound": "Nzc2MDY5", + "upper_bound": "Nzc4ODcy", + "repeats": 1 + }, + { + "count": 823588, + "lower_bound": "Nzc5MTEz", + "upper_bound": "NzgyNjMy", + "repeats": 1 + }, + { + "count": 827586, + "lower_bound": "NzgyOTY4", + "upper_bound": "Nzg2NTAz", + "repeats": 1 + }, + { + "count": 831584, + "lower_bound": "Nzg2NTI2", + "upper_bound": "NzkyMDMy", + "repeats": 1 + }, + { + "count": 835582, + "lower_bound": "NzkyMTQ1", + "upper_bound": "Nzk2NzUz", + "repeats": 1 + }, + { + "count": 839580, + "lower_bound": "Nzk3MDEz", + "upper_bound": "ODAzNjUw", + "repeats": 1 + }, + { + "count": 843578, + "lower_bound": "ODAzOTU4", + "upper_bound": "ODA4MTAy", + "repeats": 1 + }, + { + "count": 847576, + "lower_bound": "ODA4Mjgw", + "upper_bound": "ODEzMDE3", + "repeats": 1 + }, + { + "count": 851574, + "lower_bound": "ODEzNDA0", + "upper_bound": "ODE3Njcy", + "repeats": 1 + }, + { + "count": 855572, + "lower_bound": "ODE3NzM4", + "upper_bound": "ODI1MTAx", + "repeats": 1 + }, + { + "count": 859570, + "lower_bound": "ODI1MTMy", + "upper_bound": "ODI5OTA4", + "repeats": 1 + }, + { + "count": 863568, + "lower_bound": "ODMwOTky", + "upper_bound": "ODM0Mjc0", + "repeats": 1 + }, + { + "count": 867566, + "lower_bound": "ODM1MDQ4", + "upper_bound": "ODM5NjI2", + "repeats": 1 + }, + { + "count": 871564, + "lower_bound": "ODM5Njc3", + "upper_bound": "ODQzNjUw", + "repeats": 1 + }, + { + "count": 875562, + "lower_bound": "ODQ2MzA3", + "upper_bound": "ODQ5NTQz", + "repeats": 1 + }, + { + "count": 879560, + "lower_bound": "ODQ5NTg1", + "upper_bound": "ODU3MTIx", + "repeats": 1 + }, + { + "count": 883558, + "lower_bound": "ODU3NDM1", + "upper_bound": "ODYyNTMy", + "repeats": 1 + }, + { + "count": 887556, + "lower_bound": "ODYyODY3", + "upper_bound": "ODY4MjEy", + "repeats": 1 + }, + { + "count": 891554, + "lower_bound": "ODY4NjAw", + "upper_bound": "ODcxMTMz", + "repeats": 1 + }, + { + "count": 895552, + "lower_bound": "ODcxMjAx", + "upper_bound": "ODc2OTI4", + "repeats": 1 + }, + { + "count": 899550, + "lower_bound": "ODc4MTIx", + "upper_bound": "ODgyNDIw", + "repeats": 1 + }, + { + "count": 903548, + "lower_bound": "ODgzODUy", + "upper_bound": "ODg4OTkz", + "repeats": 1 + }, + { + "count": 907546, + "lower_bound": "ODg5MzI1", + "upper_bound": "ODkyOTg1", + "repeats": 1 + }, + { + "count": 911544, + "lower_bound": "ODkzMTI3", + "upper_bound": "ODk3NDE4", + "repeats": 1 + }, + { + "count": 915542, + "lower_bound": "ODk3NTc3", + "upper_bound": "OTAxNDA3", + "repeats": 1 + }, + { + "count": 919540, + "lower_bound": "OTAyNTA0", + "upper_bound": "OTA0OTM3", + "repeats": 1 + }, + { + "count": 923538, + "lower_bound": "OTA1Mzg2", + "upper_bound": "OTA5MjY3", + "repeats": 1 + }, + { + "count": 927536, + "lower_bound": "OTA5NDky", + "upper_bound": "OTEzNDg5", + "repeats": 1 + }, + { + "count": 931534, + "lower_bound": "OTEzNjE3", + "upper_bound": "OTE2MDIy", + "repeats": 1 + }, + { + "count": 935532, + "lower_bound": "OTE2MTA5", + "upper_bound": "OTIwNzcw", + "repeats": 1 + }, + { + "count": 939530, + "lower_bound": "OTIxMzQ1", + "upper_bound": "OTMwNTg5", + "repeats": 1 + }, + { + "count": 943528, + "lower_bound": "OTMwNjM1", + "upper_bound": "OTMzNTIw", + "repeats": 1 + }, + { + "count": 947526, + "lower_bound": "OTM0MTk2", + "upper_bound": "OTM5MzM1", + "repeats": 1 + }, + { + "count": 951524, + "lower_bound": "OTM5NDM4", + "upper_bound": "OTQxODY2", + "repeats": 1 + }, + { + "count": 955522, + "lower_bound": "OTQyMjM3", + "upper_bound": "OTQ4Nzk4", + "repeats": 1 + }, + { + "count": 959520, + "lower_bound": "OTUwMDQ4", + "upper_bound": "OTUyNjUy", + "repeats": 1 + }, + { + "count": 963518, + "lower_bound": "OTUyOTg3", + "upper_bound": "OTU2MDI3", + "repeats": 1 + }, + { + "count": 967516, + "lower_bound": "OTU2MjM1", + "upper_bound": "OTYwNzU2", + "repeats": 1 + }, + { + "count": 971514, + "lower_bound": "OTYxMzY3", + "upper_bound": "OTY1Mzkw", + "repeats": 1 + }, + { + "count": 975512, + "lower_bound": "OTY1NzE5", + "upper_bound": "OTY4NTQz", + "repeats": 1 + }, + { + "count": 979510, + "lower_bound": "OTY5MjE5", + "upper_bound": "OTc0Nzkx", + "repeats": 1 + }, + { + "count": 983508, + "lower_bound": "OTc1MjU4", + "upper_bound": "OTc5MjQy", + "repeats": 1 + }, + { + "count": 987506, + "lower_bound": "OTc5MzEz", + "upper_bound": "OTgyODA2", + "repeats": 1 + }, + { + "count": 991504, + "lower_bound": "OTgzMjQ5", + "upper_bound": "OTg3ODcw", + "repeats": 1 + }, + { + "count": 995502, + "lower_bound": "OTg4MTMy", + "upper_bound": "OTkxMzI0", + "repeats": 1 + }, + { + "count": 999500, + "lower_bound": "OTkxNjgx", + "upper_bound": "OTk4NDg4", + "repeats": 1 + }, + { + "count": 1000000, + "lower_bound": "OTk5MTY2", + "upper_bound": "OTk5MTY2", + "repeats": 1 + } + ] + }, + "cm_sketch": { + "rows": [ + { + "counters": [ + 495, + 474, + 474, + 471, + 509, + 474, + 509, + 527, + 490, + 509, + 475, + 503, + 506, + 489, + 472, + 504, + 513, + 469, + 508, + 480, + 476, + 529, + 488, + 486, + 513, + 477, + 507, + 477, + 514, + 533, + 456, + 512, + 460, + 474, + 473, + 489, + 472, + 517, + 459, + 516, + 489, + 491, + 497, + 492, + 472, + 508, + 498, + 496, + 438, + 472, + 485, + 452, + 484, + 493, + 477, + 471, + 534, + 500, + 466, + 499, + 476, + 458, + 493, + 481, + 497, + 473, + 479, + 490, + 469, + 519, + 471, + 483, + 482, + 489, + 506, + 544, + 504, + 492, + 457, + 506, + 479, + 477, + 491, + 464, + 496, + 509, + 492, + 471, + 440, + 479, + 476, + 505, + 522, + 498, + 442, + 471, + 492, + 499, + 516, + 494, + 477, + 493, + 500, + 478, + 496, + 482, + 508, + 500, + 495, + 490, + 456, + 501, + 541, + 469, + 507, + 497, + 477, + 450, + 515, + 472, + 501, + 500, + 514, + 480, + 459, + 476, + 470, + 491, + 493, + 520, + 535, + 508, + 494, + 549, + 501, + 489, + 528, + 511, + 488, + 449, + 455, + 452, + 482, + 474, + 514, + 472, + 445, + 476, + 505, + 484, + 466, + 489, + 510, + 495, + 489, + 479, + 467, + 521, + 480, + 500, + 501, + 495, + 457, + 457, + 487, + 487, + 502, + 477, + 472, + 498, + 473, + 489, + 486, + 480, + 531, + 480, + 500, + 510, + 505, + 499, + 477, + 473, + 506, + 475, + 494, + 496, + 447, + 490, + 500, + 474, + 513, + 512, + 470, + 480, + 480, + 539, + 476, + 506, + 503, + 481, + 526, + 493, + 449, + 500, + 467, + 497, + 461, + 497, + 502, + 468, + 475, + 497, + 501, + 510, + 469, + 477, + 439, + 497, + 469, + 494, + 492, + 482, + 495, + 470, + 438, + 512, + 485, + 525, + 460, + 541, + 500, + 476, + 502, + 510, + 476, + 482, + 520, + 435, + 447, + 487, + 487, + 461, + 485, + 461, + 488, + 531, + 519, + 476, + 444, + 490, + 510, + 506, + 492, + 465, + 507, + 512, + 517, + 492, + 481, + 490, + 488, + 487, + 453, + 490, + 536, + 494, + 515, + 504, + 504, + 510, + 511, + 489, + 478, + 461, + 522, + 503, + 495, + 508, + 469, + 454, + 522, + 537, + 482, + 506, + 493, + 501, + 513, + 450, + 493, + 476, + 499, + 491, + 468, + 504, + 524, + 488, + 469, + 478, + 457, + 473, + 481, + 469, + 479, + 491, + 488, + 483, + 509, + 523, + 517, + 497, + 520, + 491, + 453, + 520, + 508, + 486, + 488, + 489, + 543, + 469, + 498, + 471, + 472, + 495, + 507, + 473, + 497, + 487, + 493, + 462, + 475, + 474, + 490, + 518, + 492, + 462, + 488, + 497, + 474, + 502, + 550, + 490, + 452, + 492, + 465, + 488, + 467, + 490, + 513, + 448, + 510, + 490, + 459, + 477, + 487, + 500, + 501, + 505, + 491, + 506, + 443, + 482, + 466, + 482, + 459, + 461, + 523, + 502, + 466, + 469, + 504, + 494, + 483, + 440, + 497, + 499, + 483, + 500, + 484, + 500, + 527, + 454, + 493, + 445, + 474, + 486, + 507, + 490, + 479, + 493, + 459, + 474, + 492, + 482, + 497, + 483, + 516, + 499, + 462, + 522, + 482, + 498, + 486, + 464, + 504, + 496, + 498, + 493, + 481, + 462, + 508, + 484, + 485, + 479, + 487, + 489, + 493, + 471, + 495, + 541, + 495, + 472, + 482, + 461, + 471, + 501, + 480, + 460, + 503, + 500, + 511, + 492, + 491, + 450, + 520, + 462, + 510, + 499, + 460, + 515, + 519, + 455, + 495, + 470, + 464, + 509, + 479, + 456, + 466, + 523, + 478, + 461, + 496, + 500, + 469, + 506, + 512, + 490, + 523, + 485, + 452, + 528, + 486, + 458, + 468, + 476, + 510, + 500, + 499, + 485, + 478, + 526, + 526, + 493, + 478, + 491, + 503, + 467, + 506, + 478, + 501, + 453, + 520, + 499, + 469, + 479, + 479, + 497, + 483, + 492, + 476, + 511, + 469, + 510, + 508, + 457, + 482, + 475, + 504, + 462, + 508, + 455, + 518, + 516, + 485, + 505, + 478, + 504, + 476, + 486, + 484, + 495, + 510, + 476, + 491, + 469, + 450, + 472, + 497, + 470, + 491, + 504, + 469, + 479, + 481, + 521, + 470, + 524, + 508, + 487, + 466, + 427, + 544, + 489, + 491, + 471, + 505, + 497, + 469, + 478, + 490, + 461, + 462, + 446, + 487, + 492, + 493, + 484, + 521, + 545, + 457, + 490, + 470, + 471, + 522, + 494, + 483, + 468, + 487, + 490, + 472, + 470, + 471, + 484, + 488, + 500, + 496, + 465, + 483, + 477, + 499, + 465, + 446, + 495, + 472, + 468, + 461, + 513, + 493, + 467, + 481, + 488, + 481, + 507, + 503, + 494, + 500, + 487, + 492, + 516, + 484, + 433, + 524, + 507, + 522, + 506, + 482, + 464, + 503, + 460, + 502, + 511, + 490, + 517, + 520, + 523, + 520, + 478, + 486, + 511, + 508, + 487, + 495, + 511, + 466, + 478, + 504, + 485, + 480, + 483, + 482, + 469, + 488, + 513, + 517, + 501, + 452, + 482, + 466, + 471, + 460, + 476, + 478, + 514, + 487, + 473, + 507, + 520, + 487, + 484, + 484, + 487, + 468, + 465, + 439, + 469, + 485, + 494, + 488, + 508, + 477, + 495, + 465, + 525, + 502, + 501, + 475, + 487, + 456, + 512, + 515, + 476, + 444, + 470, + 526, + 521, + 550, + 517, + 523, + 456, + 465, + 486, + 477, + 475, + 479, + 481, + 513, + 550, + 505, + 488, + 467, + 518, + 493, + 485, + 452, + 506, + 483, + 495, + 497, + 500, + 465, + 502, + 500, + 468, + 480, + 491, + 487, + 461, + 490, + 452, + 467, + 462, + 472, + 510, + 477, + 494, + 481, + 500, + 476, + 494, + 485, + 496, + 520, + 474, + 455, + 484, + 517, + 492, + 506, + 498, + 495, + 518, + 466, + 467, + 487, + 511, + 506, + 523, + 481, + 453, + 475, + 472, + 501, + 511, + 480, + 509, + 475, + 466, + 479, + 464, + 492, + 485, + 455, + 490, + 478, + 482, + 461, + 481, + 501, + 506, + 479, + 498, + 485, + 451, + 510, + 475, + 482, + 473, + 459, + 443, + 496, + 449, + 447, + 459, + 451, + 527, + 469, + 509, + 464, + 450, + 476, + 496, + 494, + 493, + 479, + 532, + 505, + 487, + 425, + 462, + 484, + 475, + 478, + 510, + 486, + 509, + 479, + 498, + 522, + 552, + 449, + 507, + 531, + 514, + 454, + 476, + 496, + 482, + 494, + 479, + 519, + 500, + 495, + 536, + 447, + 487, + 533, + 524, + 495, + 466, + 475, + 547, + 525, + 485, + 499, + 477, + 457, + 515, + 447, + 489, + 435, + 480, + 485, + 492, + 483, + 496, + 458, + 447, + 475, + 491, + 487, + 481, + 489, + 482, + 487, + 489, + 485, + 485, + 458, + 480, + 474, + 496, + 523, + 491, + 457, + 488, + 503, + 490, + 525, + 451, + 502, + 481, + 499, + 478, + 491, + 510, + 487, + 500, + 493, + 458, + 485, + 504, + 500, + 455, + 519, + 478, + 477, + 486, + 481, + 488, + 499, + 492, + 510, + 489, + 487, + 493, + 452, + 505, + 464, + 489, + 476, + 463, + 483, + 471, + 468, + 475, + 447, + 468, + 448, + 502, + 481, + 489, + 512, + 507, + 451, + 475, + 502, + 484, + 496, + 517, + 482, + 499, + 453, + 515, + 481, + 474, + 518, + 493, + 462, + 481, + 507, + 507, + 476, + 474, + 484, + 487, + 484, + 537, + 508, + 552, + 486, + 476, + 500, + 481, + 458, + 508, + 510, + 474, + 469, + 518, + 449, + 462, + 511, + 466, + 459, + 507, + 474, + 448, + 467, + 484, + 519, + 478, + 481, + 501, + 507, + 485, + 483, + 489, + 477, + 517, + 504, + 511, + 468, + 506, + 445, + 529, + 507, + 518, + 467, + 494, + 493, + 464, + 499, + 488, + 484, + 490, + 515, + 525, + 492, + 463, + 494, + 454, + 528, + 455, + 479, + 470, + 474, + 491, + 509, + 479, + 477, + 450, + 485, + 492, + 494, + 468, + 454, + 513, + 488, + 463, + 425, + 486, + 498, + 517, + 457, + 521, + 459, + 508, + 526, + 490, + 500, + 457, + 450, + 484, + 504, + 511, + 512, + 447, + 542, + 430, + 496, + 482, + 485, + 501, + 487, + 486, + 477, + 471, + 541, + 456, + 484, + 514, + 503, + 488, + 480, + 489, + 484, + 470, + 468, + 467, + 491, + 466, + 492, + 516, + 479, + 510, + 522, + 485, + 473, + 488, + 477, + 482, + 448, + 475, + 528, + 476, + 459, + 529, + 492, + 501, + 479, + 507, + 483, + 435, + 486, + 537, + 455, + 496, + 489, + 466, + 508, + 494, + 507, + 502, + 483, + 501, + 498, + 552, + 463, + 519, + 460, + 496, + 456, + 475, + 519, + 491, + 506, + 502, + 491, + 498, + 435, + 500, + 514, + 484, + 490, + 513, + 492, + 461, + 505, + 488, + 455, + 465, + 512, + 495, + 438, + 481, + 494, + 501, + 472, + 461, + 527, + 521, + 505, + 479, + 512, + 516, + 485, + 506, + 493, + 460, + 447, + 469, + 502, + 472, + 470, + 499, + 483, + 477, + 465, + 502, + 509, + 492, + 468, + 504, + 457, + 519, + 477, + 503, + 485, + 479, + 508, + 481, + 517, + 477, + 480, + 474, + 473, + 477, + 463, + 471, + 504, + 495, + 481, + 490, + 515, + 472, + 478, + 446, + 483, + 507, + 452, + 461, + 504, + 491, + 514, + 479, + 516, + 479, + 476, + 491, + 491, + 494, + 513, + 505, + 491, + 473, + 518, + 512, + 497, + 515, + 442, + 497, + 513, + 453, + 482, + 471, + 549, + 459, + 507, + 516, + 460, + 496, + 503, + 495, + 474, + 493, + 464, + 553, + 486, + 497, + 468, + 473, + 508, + 551, + 547, + 438, + 467, + 483, + 488, + 463, + 479, + 500, + 517, + 497, + 498, + 498, + 491, + 456, + 472, + 525, + 533, + 479, + 491, + 480, + 450, + 505, + 463, + 487, + 451, + 495, + 512, + 466, + 498, + 498, + 510, + 523, + 496, + 478, + 468, + 513, + 508, + 512, + 465, + 509, + 500, + 500, + 488, + 504, + 459, + 495, + 463, + 515, + 480, + 510, + 495, + 468, + 487, + 514, + 487, + 479, + 447, + 492, + 490, + 508, + 457, + 476, + 503, + 485, + 486, + 535, + 462, + 510, + 474, + 475, + 484, + 487, + 515, + 485, + 493, + 502, + 456, + 486, + 543, + 509, + 500, + 461, + 504, + 503, + 447, + 488, + 445, + 458, + 458, + 508, + 466, + 477, + 479, + 503, + 453, + 511, + 535, + 532, + 497, + 490, + 495, + 496, + 472, + 451, + 502, + 482, + 479, + 468, + 464, + 484, + 537, + 439, + 474, + 482, + 502, + 472, + 492, + 517, + 488, + 481, + 506, + 485, + 486, + 459, + 468, + 478, + 446, + 464, + 506, + 434, + 558, + 491, + 514, + 474, + 482, + 472, + 471, + 484, + 523, + 473, + 467, + 518, + 516, + 443, + 457, + 513, + 451, + 483, + 485, + 501, + 491, + 490, + 480, + 457, + 493, + 493, + 518, + 521, + 485, + 481, + 504, + 495, + 491, + 483, + 514, + 500, + 460, + 469, + 514, + 484, + 480, + 515, + 498, + 494, + 496, + 506, + 487, + 480, + 483, + 503, + 496, + 492, + 432, + 442, + 482, + 514, + 492, + 485, + 488, + 467, + 480, + 458, + 482, + 529, + 528, + 456, + 528, + 485, + 517, + 457, + 501, + 490, + 471, + 455, + 513, + 460, + 502, + 505, + 498, + 476, + 508, + 484, + 500, + 487, + 499, + 478, + 473, + 483, + 500, + 496, + 485, + 506, + 450, + 487, + 537, + 470, + 461, + 486, + 492, + 511, + 473, + 482, + 511, + 484, + 494, + 492, + 485, + 498, + 511, + 477, + 488, + 487, + 474, + 500, + 454, + 515, + 477, + 507, + 505, + 485, + 527, + 456, + 493, + 484, + 509, + 526, + 473, + 458, + 493, + 524, + 511, + 497, + 481, + 504, + 484, + 486, + 489, + 477, + 515, + 489, + 520, + 506, + 475, + 517, + 465, + 490, + 504, + 480, + 461, + 504, + 524, + 477, + 465, + 507, + 471, + 486, + 503, + 453, + 481, + 493, + 490, + 482, + 495, + 508, + 482, + 431, + 487, + 503, + 479, + 495, + 493, + 501, + 515, + 516, + 469, + 460, + 448, + 530, + 468, + 510, + 472, + 491, + 464, + 501, + 502, + 483, + 508, + 499, + 494, + 456, + 425, + 489, + 548, + 484, + 489, + 505, + 481, + 497, + 495, + 465, + 490, + 475, + 494, + 505, + 530, + 479, + 506, + 489, + 529, + 462, + 502, + 478, + 487, + 464, + 471, + 477, + 496, + 479, + 462, + 500, + 507, + 481, + 489, + 509, + 519, + 504, + 511, + 470, + 428, + 482, + 526, + 502, + 465, + 448, + 460, + 449, + 509, + 480, + 491, + 493, + 485, + 461, + 474, + 432, + 488, + 508, + 500, + 451, + 468, + 473, + 501, + 495, + 454, + 510, + 478, + 506, + 511, + 517, + 463, + 477, + 466, + 517, + 478, + 464, + 477, + 493, + 526, + 489, + 471, + 511, + 488, + 499, + 512, + 484, + 494, + 544, + 482, + 487, + 492, + 506, + 520, + 484, + 493, + 485, + 504, + 512, + 469, + 478, + 480, + 504, + 511, + 487, + 509, + 480, + 502, + 464, + 468, + 486, + 484, + 493, + 532, + 441, + 520, + 499, + 523, + 467, + 462, + 488, + 482, + 470, + 511, + 532, + 490, + 515, + 490, + 487, + 479, + 485, + 505, + 504, + 530, + 434, + 472, + 488, + 485, + 506, + 456, + 500, + 533, + 467, + 513, + 455, + 442, + 470, + 521, + 479, + 492, + 477, + 490, + 501, + 478, + 503, + 469, + 476, + 440, + 482, + 484, + 471, + 511, + 492, + 513, + 524, + 493, + 505, + 466, + 515, + 495, + 466, + 466, + 490, + 471, + 481, + 468, + 492, + 519, + 457, + 464, + 507, + 528, + 468, + 465, + 517, + 475, + 488, + 481, + 490, + 472, + 430, + 477, + 516, + 498, + 508, + 489, + 487, + 522, + 478, + 473, + 538, + 533, + 497, + 475, + 482, + 521, + 504, + 503, + 494, + 503, + 467, + 467, + 475, + 462, + 495, + 526, + 495, + 478, + 491, + 459, + 493, + 491, + 534, + 531, + 478, + 502, + 490, + 473, + 477, + 475, + 476, + 494, + 505, + 487, + 462, + 494, + 468, + 502, + 433, + 523, + 450, + 495, + 455, + 479, + 506, + 488, + 500, + 499, + 498, + 463, + 531, + 488, + 477, + 455, + 480, + 480, + 514, + 507, + 483, + 495, + 521, + 491, + 497, + 510, + 524, + 463, + 468, + 501, + 450, + 492, + 456, + 498, + 471, + 500, + 491, + 497, + 501, + 479, + 502, + 526, + 484, + 461, + 487, + 533, + 464, + 468, + 494, + 487, + 465, + 471, + 506, + 510, + 451, + 501, + 478, + 458, + 496, + 502, + 492, + 512, + 482, + 469, + 491, + 519, + 519, + 491, + 518, + 507, + 463, + 473, + 487, + 507, + 442, + 484, + 446, + 450, + 454, + 498, + 482, + 487, + 492, + 484, + 548, + 494, + 470, + 458, + 503, + 481, + 492, + 488, + 531, + 487, + 450, + 468, + 440, + 476, + 460, + 474, + 497, + 526, + 518, + 489, + 501, + 524, + 503, + 464, + 464, + 537, + 489, + 469, + 500, + 525, + 469, + 479, + 491, + 450, + 523, + 444, + 487, + 458, + 472, + 514, + 473, + 493, + 480, + 486, + 517, + 510, + 458, + 468, + 480, + 467, + 513, + 530, + 505, + 461, + 436, + 491, + 472, + 479, + 531, + 493, + 468, + 464, + 540, + 493, + 502, + 496, + 490, + 490, + 462, + 513, + 506, + 504, + 497, + 508, + 503, + 477, + 500, + 512, + 490, + 480, + 496, + 467, + 517, + 473, + 476, + 486, + 505, + 498, + 532, + 513, + 452, + 498, + 448, + 508, + 526, + 488, + 488, + 488, + 479, + 542, + 493, + 507, + 496, + 491, + 493, + 482, + 441, + 462, + 503, + 512, + 475, + 500, + 508, + 496, + 506, + 470, + 477, + 511, + 480, + 452, + 453, + 489, + 473, + 507, + 488, + 473, + 494, + 492, + 489, + 491, + 457, + 481, + 493, + 494, + 451, + 503, + 489, + 475, + 477, + 473, + 494, + 499, + 499, + 475, + 478, + 488, + 489, + 475, + 451, + 512, + 525, + 514, + 504, + 480, + 482, + 486, + 481, + 480, + 497, + 465, + 503, + 473, + 465, + 487, + 468, + 481, + 499, + 450, + 496, + 521, + 447, + 462, + 485, + 519, + 498, + 491, + 462, + 489, + 492, + 485, + 473, + 519, + 503, + 540, + 539, + 482, + 482, + 531, + 532, + 461, + 473, + 542, + 503, + 532, + 420, + 490, + 453, + 491, + 471, + 477, + 470, + 490, + 485, + 499, + 482, + 500, + 517, + 489, + 500, + 511, + 452, + 453, + 486, + 452, + 502 + ] + }, + { + "counters": [ + 492, + 472, + 474, + 474, + 468, + 472, + 473, + 512, + 480, + 500, + 481, + 508, + 468, + 466, + 461, + 493, + 492, + 523, + 494, + 459, + 479, + 491, + 473, + 484, + 489, + 488, + 479, + 499, + 497, + 444, + 560, + 452, + 474, + 519, + 463, + 517, + 499, + 484, + 471, + 490, + 496, + 476, + 454, + 478, + 459, + 484, + 473, + 467, + 494, + 469, + 497, + 477, + 558, + 494, + 504, + 478, + 518, + 498, + 483, + 507, + 534, + 497, + 473, + 479, + 485, + 477, + 477, + 487, + 477, + 442, + 480, + 499, + 498, + 500, + 503, + 486, + 480, + 490, + 466, + 482, + 468, + 455, + 505, + 483, + 466, + 468, + 497, + 485, + 513, + 466, + 524, + 491, + 482, + 543, + 439, + 521, + 517, + 441, + 451, + 492, + 524, + 479, + 480, + 504, + 486, + 473, + 481, + 532, + 449, + 515, + 492, + 485, + 510, + 508, + 471, + 484, + 466, + 549, + 499, + 491, + 498, + 511, + 519, + 496, + 512, + 502, + 518, + 469, + 466, + 495, + 466, + 511, + 462, + 505, + 482, + 511, + 493, + 464, + 435, + 509, + 503, + 508, + 473, + 464, + 446, + 490, + 479, + 471, + 450, + 490, + 504, + 493, + 494, + 493, + 465, + 446, + 473, + 431, + 497, + 471, + 476, + 498, + 463, + 510, + 478, + 536, + 488, + 512, + 473, + 486, + 510, + 529, + 485, + 501, + 522, + 461, + 476, + 518, + 491, + 496, + 492, + 474, + 441, + 481, + 493, + 514, + 483, + 510, + 499, + 421, + 484, + 507, + 506, + 454, + 466, + 460, + 475, + 503, + 461, + 486, + 494, + 494, + 503, + 460, + 496, + 465, + 467, + 517, + 498, + 483, + 479, + 469, + 492, + 507, + 491, + 487, + 473, + 506, + 510, + 489, + 489, + 447, + 486, + 481, + 497, + 473, + 497, + 519, + 479, + 514, + 499, + 475, + 470, + 509, + 474, + 458, + 473, + 474, + 479, + 510, + 500, + 466, + 461, + 474, + 513, + 496, + 477, + 501, + 499, + 483, + 493, + 468, + 519, + 467, + 432, + 488, + 476, + 507, + 458, + 515, + 492, + 455, + 481, + 508, + 487, + 524, + 488, + 441, + 447, + 422, + 481, + 501, + 526, + 476, + 471, + 512, + 499, + 451, + 515, + 500, + 523, + 487, + 492, + 462, + 485, + 508, + 479, + 477, + 487, + 477, + 484, + 449, + 477, + 498, + 453, + 495, + 456, + 434, + 463, + 480, + 494, + 501, + 495, + 487, + 522, + 468, + 469, + 515, + 515, + 472, + 523, + 488, + 473, + 481, + 471, + 488, + 470, + 442, + 497, + 516, + 496, + 488, + 498, + 542, + 469, + 431, + 502, + 431, + 484, + 499, + 530, + 467, + 477, + 482, + 473, + 505, + 473, + 497, + 467, + 482, + 495, + 486, + 522, + 456, + 505, + 482, + 520, + 479, + 527, + 510, + 499, + 477, + 487, + 422, + 478, + 495, + 466, + 474, + 496, + 490, + 442, + 483, + 489, + 472, + 483, + 511, + 505, + 469, + 497, + 479, + 507, + 516, + 461, + 491, + 504, + 481, + 500, + 483, + 517, + 478, + 472, + 529, + 514, + 491, + 533, + 485, + 475, + 493, + 492, + 472, + 490, + 520, + 479, + 468, + 506, + 502, + 464, + 464, + 476, + 461, + 512, + 518, + 497, + 507, + 495, + 477, + 489, + 457, + 493, + 455, + 478, + 503, + 428, + 519, + 470, + 481, + 472, + 492, + 478, + 483, + 466, + 479, + 481, + 485, + 513, + 486, + 454, + 497, + 487, + 476, + 516, + 501, + 458, + 459, + 471, + 479, + 546, + 499, + 464, + 452, + 437, + 496, + 494, + 507, + 481, + 476, + 464, + 497, + 493, + 512, + 503, + 472, + 491, + 496, + 495, + 492, + 502, + 521, + 482, + 484, + 527, + 476, + 498, + 454, + 453, + 526, + 519, + 484, + 482, + 522, + 511, + 519, + 471, + 493, + 486, + 507, + 509, + 504, + 513, + 461, + 518, + 524, + 475, + 469, + 454, + 492, + 536, + 464, + 531, + 456, + 463, + 478, + 468, + 487, + 483, + 534, + 489, + 471, + 470, + 517, + 473, + 470, + 476, + 524, + 490, + 506, + 525, + 499, + 493, + 481, + 466, + 479, + 492, + 482, + 498, + 502, + 477, + 488, + 482, + 505, + 488, + 473, + 474, + 465, + 457, + 513, + 502, + 481, + 519, + 521, + 507, + 524, + 490, + 506, + 500, + 507, + 461, + 494, + 472, + 510, + 513, + 459, + 480, + 508, + 481, + 458, + 512, + 478, + 488, + 501, + 475, + 473, + 482, + 467, + 519, + 476, + 465, + 511, + 510, + 517, + 455, + 461, + 462, + 488, + 500, + 479, + 450, + 540, + 471, + 485, + 507, + 466, + 465, + 454, + 483, + 448, + 493, + 492, + 478, + 472, + 485, + 495, + 492, + 484, + 482, + 473, + 440, + 498, + 483, + 465, + 479, + 454, + 467, + 485, + 506, + 450, + 479, + 482, + 533, + 479, + 483, + 475, + 459, + 498, + 481, + 481, + 435, + 488, + 486, + 469, + 499, + 488, + 475, + 497, + 479, + 490, + 469, + 508, + 499, + 514, + 517, + 498, + 521, + 511, + 473, + 529, + 505, + 495, + 485, + 497, + 501, + 498, + 485, + 514, + 492, + 531, + 460, + 502, + 450, + 488, + 511, + 499, + 487, + 491, + 498, + 480, + 478, + 515, + 479, + 472, + 489, + 447, + 495, + 505, + 459, + 466, + 489, + 465, + 475, + 494, + 500, + 510, + 490, + 517, + 497, + 494, + 478, + 501, + 469, + 497, + 507, + 465, + 506, + 491, + 466, + 493, + 487, + 520, + 454, + 552, + 472, + 451, + 490, + 507, + 498, + 510, + 524, + 485, + 503, + 462, + 514, + 432, + 505, + 474, + 481, + 454, + 497, + 477, + 514, + 502, + 510, + 523, + 471, + 495, + 465, + 488, + 498, + 499, + 506, + 522, + 454, + 474, + 465, + 501, + 453, + 495, + 474, + 458, + 506, + 536, + 494, + 455, + 474, + 463, + 478, + 465, + 503, + 474, + 461, + 499, + 521, + 486, + 478, + 501, + 516, + 470, + 439, + 474, + 516, + 507, + 541, + 509, + 492, + 461, + 507, + 496, + 499, + 496, + 468, + 472, + 461, + 477, + 484, + 510, + 504, + 490, + 466, + 485, + 483, + 496, + 484, + 512, + 483, + 521, + 500, + 477, + 479, + 500, + 476, + 543, + 478, + 505, + 483, + 471, + 495, + 484, + 484, + 475, + 512, + 476, + 478, + 469, + 497, + 475, + 474, + 485, + 475, + 480, + 478, + 475, + 498, + 467, + 518, + 502, + 493, + 488, + 495, + 491, + 526, + 477, + 524, + 491, + 438, + 485, + 463, + 492, + 484, + 509, + 494, + 509, + 466, + 493, + 500, + 492, + 486, + 457, + 475, + 490, + 542, + 486, + 488, + 489, + 498, + 506, + 521, + 507, + 516, + 494, + 478, + 518, + 516, + 489, + 517, + 497, + 445, + 485, + 484, + 455, + 509, + 478, + 510, + 486, + 507, + 491, + 495, + 484, + 498, + 491, + 534, + 429, + 489, + 521, + 488, + 475, + 489, + 464, + 482, + 531, + 473, + 475, + 483, + 462, + 492, + 505, + 484, + 493, + 483, + 453, + 498, + 451, + 490, + 530, + 516, + 466, + 485, + 486, + 433, + 481, + 471, + 526, + 520, + 447, + 519, + 514, + 449, + 489, + 552, + 508, + 476, + 506, + 477, + 482, + 459, + 486, + 463, + 496, + 457, + 456, + 446, + 466, + 459, + 519, + 465, + 465, + 481, + 458, + 494, + 492, + 496, + 500, + 485, + 481, + 507, + 478, + 544, + 477, + 479, + 479, + 501, + 469, + 456, + 493, + 495, + 465, + 517, + 428, + 465, + 514, + 494, + 470, + 505, + 488, + 484, + 483, + 498, + 493, + 493, + 519, + 492, + 494, + 494, + 484, + 470, + 484, + 490, + 510, + 456, + 473, + 502, + 482, + 487, + 470, + 458, + 474, + 499, + 472, + 479, + 514, + 429, + 475, + 493, + 507, + 501, + 521, + 520, + 490, + 513, + 498, + 495, + 507, + 494, + 487, + 498, + 499, + 535, + 531, + 524, + 446, + 486, + 450, + 455, + 468, + 484, + 500, + 509, + 499, + 502, + 489, + 526, + 501, + 515, + 476, + 490, + 510, + 492, + 440, + 468, + 487, + 430, + 514, + 475, + 460, + 475, + 504, + 482, + 451, + 504, + 512, + 464, + 486, + 502, + 513, + 480, + 508, + 467, + 483, + 446, + 511, + 480, + 492, + 496, + 528, + 505, + 485, + 469, + 436, + 529, + 468, + 456, + 493, + 479, + 499, + 493, + 500, + 487, + 528, + 475, + 499, + 480, + 505, + 482, + 512, + 519, + 493, + 499, + 478, + 495, + 479, + 498, + 520, + 480, + 485, + 506, + 465, + 518, + 479, + 511, + 468, + 505, + 472, + 492, + 500, + 523, + 492, + 494, + 470, + 535, + 471, + 455, + 463, + 505, + 477, + 499, + 484, + 508, + 490, + 493, + 484, + 492, + 523, + 510, + 471, + 487, + 511, + 467, + 478, + 484, + 504, + 488, + 496, + 471, + 443, + 487, + 499, + 515, + 504, + 503, + 514, + 516, + 472, + 491, + 461, + 465, + 499, + 464, + 487, + 479, + 475, + 511, + 482, + 468, + 467, + 499, + 524, + 494, + 516, + 470, + 483, + 488, + 485, + 453, + 474, + 475, + 516, + 466, + 486, + 515, + 464, + 477, + 489, + 471, + 492, + 500, + 462, + 495, + 460, + 482, + 488, + 476, + 478, + 488, + 496, + 516, + 485, + 503, + 498, + 507, + 471, + 506, + 495, + 503, + 506, + 479, + 464, + 478, + 507, + 446, + 506, + 491, + 478, + 463, + 460, + 478, + 495, + 504, + 521, + 497, + 491, + 495, + 491, + 484, + 474, + 500, + 497, + 509, + 492, + 470, + 504, + 509, + 499, + 555, + 485, + 462, + 465, + 469, + 523, + 485, + 471, + 519, + 493, + 480, + 456, + 492, + 490, + 468, + 496, + 520, + 479, + 500, + 482, + 516, + 475, + 505, + 471, + 488, + 459, + 483, + 500, + 518, + 496, + 492, + 488, + 479, + 526, + 436, + 497, + 479, + 474, + 488, + 526, + 513, + 497, + 508, + 484, + 522, + 495, + 465, + 499, + 497, + 482, + 487, + 496, + 497, + 500, + 532, + 500, + 506, + 484, + 448, + 462, + 446, + 494, + 475, + 477, + 515, + 519, + 494, + 505, + 489, + 560, + 481, + 515, + 489, + 483, + 558, + 446, + 480, + 445, + 489, + 512, + 453, + 523, + 515, + 469, + 434, + 480, + 511, + 479, + 472, + 481, + 473, + 466, + 454, + 506, + 456, + 492, + 468, + 518, + 534, + 530, + 520, + 525, + 495, + 463, + 466, + 492, + 482, + 516, + 506, + 445, + 478, + 502, + 487, + 478, + 495, + 501, + 503, + 480, + 511, + 463, + 490, + 485, + 479, + 496, + 518, + 475, + 492, + 464, + 460, + 487, + 485, + 491, + 509, + 449, + 492, + 501, + 485, + 526, + 523, + 451, + 495, + 460, + 480, + 513, + 520, + 496, + 538, + 499, + 484, + 481, + 480, + 514, + 467, + 472, + 476, + 494, + 495, + 467, + 485, + 511, + 476, + 495, + 476, + 506, + 496, + 496, + 525, + 480, + 522, + 523, + 489, + 506, + 467, + 507, + 489, + 479, + 523, + 479, + 494, + 437, + 470, + 499, + 508, + 484, + 495, + 478, + 462, + 482, + 539, + 516, + 492, + 499, + 508, + 493, + 470, + 484, + 496, + 492, + 498, + 451, + 496, + 497, + 479, + 456, + 458, + 466, + 468, + 535, + 498, + 527, + 509, + 505, + 522, + 506, + 465, + 486, + 425, + 530, + 502, + 514, + 483, + 524, + 464, + 460, + 512, + 494, + 500, + 493, + 459, + 487, + 502, + 491, + 498, + 485, + 496, + 516, + 467, + 485, + 519, + 481, + 501, + 487, + 483, + 476, + 489, + 511, + 463, + 471, + 486, + 508, + 536, + 488, + 449, + 481, + 503, + 480, + 472, + 489, + 459, + 543, + 465, + 483, + 467, + 460, + 508, + 492, + 441, + 534, + 497, + 459, + 506, + 483, + 479, + 510, + 484, + 468, + 488, + 475, + 518, + 491, + 480, + 533, + 513, + 460, + 560, + 484, + 481, + 476, + 497, + 503, + 484, + 492, + 455, + 485, + 473, + 502, + 528, + 485, + 524, + 492, + 469, + 500, + 447, + 476, + 501, + 494, + 498, + 481, + 505, + 499, + 502, + 466, + 490, + 489, + 500, + 464, + 504, + 495, + 532, + 440, + 483, + 506, + 472, + 483, + 496, + 474, + 481, + 507, + 489, + 480, + 502, + 486, + 510, + 448, + 524, + 493, + 472, + 512, + 492, + 490, + 458, + 464, + 460, + 469, + 505, + 484, + 479, + 511, + 503, + 480, + 490, + 473, + 517, + 470, + 499, + 492, + 512, + 455, + 487, + 483, + 495, + 505, + 500, + 463, + 492, + 515, + 494, + 507, + 513, + 520, + 466, + 488, + 473, + 466, + 467, + 445, + 438, + 492, + 465, + 492, + 540, + 451, + 521, + 480, + 475, + 445, + 471, + 466, + 472, + 482, + 512, + 471, + 490, + 487, + 501, + 473, + 445, + 506, + 470, + 500, + 471, + 457, + 531, + 508, + 477, + 498, + 514, + 477, + 494, + 480, + 484, + 525, + 477, + 497, + 498, + 520, + 519, + 485, + 481, + 528, + 484, + 498, + 523, + 504, + 478, + 487, + 489, + 482, + 481, + 486, + 488, + 499, + 461, + 482, + 490, + 473, + 496, + 467, + 461, + 459, + 496, + 490, + 484, + 486, + 503, + 480, + 473, + 449, + 462, + 471, + 518, + 494, + 486, + 482, + 523, + 466, + 466, + 475, + 491, + 489, + 458, + 472, + 461, + 495, + 501, + 507, + 441, + 460, + 461, + 474, + 473, + 465, + 483, + 494, + 491, + 494, + 487, + 497, + 523, + 464, + 508, + 505, + 465, + 475, + 485, + 457, + 467, + 470, + 506, + 482, + 469, + 512, + 463, + 522, + 455, + 454, + 476, + 504, + 502, + 505, + 519, + 509, + 488, + 498, + 522, + 493, + 462, + 461, + 489, + 468, + 506, + 506, + 461, + 460, + 466, + 494, + 472, + 481, + 520, + 479, + 474, + 470, + 503, + 526, + 505, + 482, + 494, + 518, + 504, + 478, + 478, + 515, + 504, + 494, + 462, + 501, + 489, + 508, + 494, + 517, + 478, + 480, + 456, + 500, + 482, + 490, + 449, + 490, + 517, + 490, + 475, + 467, + 468, + 509, + 471, + 459, + 491, + 503, + 511, + 494, + 468, + 482, + 492, + 498, + 464, + 503, + 469, + 500, + 531, + 511, + 500, + 489, + 504, + 492, + 485, + 501, + 501, + 508, + 474, + 474, + 451, + 487, + 505, + 497, + 493, + 485, + 468, + 479, + 490, + 520, + 466, + 507, + 451, + 479, + 469, + 565, + 488, + 473, + 493, + 518, + 488, + 534, + 471, + 452, + 524, + 470, + 517, + 494, + 496, + 505, + 486, + 493, + 489, + 497, + 495, + 503, + 513, + 516, + 482, + 513, + 455, + 470, + 466, + 527, + 495, + 490, + 476, + 492, + 454, + 500, + 481, + 493, + 492, + 512, + 451, + 481, + 528, + 510, + 479, + 499, + 499, + 494, + 478, + 512, + 476, + 483, + 504, + 531, + 511, + 453, + 460, + 496, + 467, + 463, + 467, + 531, + 485, + 455, + 458, + 504, + 477, + 492, + 481, + 501, + 499, + 462, + 493, + 480, + 504, + 517, + 476, + 487, + 484, + 494, + 512, + 521, + 481, + 453, + 470, + 461, + 519, + 479, + 495, + 464, + 466, + 468, + 507, + 503, + 493, + 474, + 482, + 508, + 473, + 484, + 479, + 487, + 476, + 485, + 460, + 449, + 507, + 506, + 478, + 471, + 481, + 477, + 479, + 507, + 472, + 484, + 464, + 504, + 512, + 510, + 497, + 482, + 484, + 511, + 480, + 533, + 512, + 523, + 508, + 502, + 493, + 473, + 514, + 531, + 505, + 491, + 483, + 435, + 512, + 488, + 495, + 501, + 479, + 459, + 495, + 487, + 476, + 478, + 455, + 503, + 482, + 508, + 469, + 532, + 500, + 502, + 468, + 495, + 489, + 532, + 496, + 521, + 471, + 494, + 464, + 510, + 500, + 476, + 528, + 498, + 496, + 504, + 487, + 481, + 482, + 531, + 483, + 509, + 499, + 540, + 514, + 518, + 510, + 465, + 467, + 475, + 502, + 498, + 473, + 488, + 514, + 491, + 491, + 435, + 469, + 470, + 471, + 460, + 483, + 477, + 530, + 496, + 468, + 448, + 479, + 511, + 507, + 453, + 515, + 467, + 476, + 513, + 481, + 469, + 493, + 467, + 454, + 487, + 468, + 492, + 468, + 471, + 469, + 472, + 458, + 450, + 456, + 485, + 542, + 508, + 483, + 501, + 485, + 512, + 481, + 512, + 509, + 521, + 489, + 495, + 478, + 529, + 478, + 504, + 473, + 465, + 529, + 482, + 500, + 483, + 465, + 470, + 478, + 507, + 507, + 473, + 504, + 485, + 461, + 517, + 483, + 489, + 526, + 487, + 477, + 485, + 501, + 444, + 530, + 504, + 470, + 463, + 489, + 497 + ] + }, + { + "counters": [ + 486, + 470, + 490, + 485, + 472, + 508, + 468, + 498, + 490, + 487, + 516, + 485, + 486, + 519, + 497, + 503, + 474, + 490, + 491, + 477, + 483, + 473, + 474, + 478, + 483, + 459, + 503, + 478, + 524, + 464, + 472, + 486, + 492, + 449, + 455, + 486, + 458, + 477, + 491, + 482, + 493, + 505, + 498, + 506, + 516, + 450, + 499, + 455, + 469, + 466, + 542, + 548, + 519, + 482, + 487, + 479, + 491, + 477, + 505, + 465, + 478, + 460, + 452, + 488, + 470, + 481, + 522, + 519, + 471, + 470, + 528, + 499, + 518, + 511, + 493, + 521, + 489, + 477, + 478, + 519, + 502, + 511, + 490, + 466, + 494, + 446, + 503, + 468, + 468, + 467, + 489, + 504, + 474, + 508, + 500, + 449, + 486, + 490, + 530, + 467, + 481, + 485, + 511, + 488, + 521, + 470, + 508, + 525, + 510, + 504, + 511, + 502, + 513, + 461, + 501, + 485, + 476, + 504, + 489, + 486, + 488, + 497, + 541, + 482, + 487, + 494, + 482, + 499, + 506, + 452, + 484, + 495, + 470, + 453, + 482, + 489, + 494, + 485, + 475, + 491, + 481, + 498, + 523, + 493, + 483, + 505, + 493, + 503, + 475, + 491, + 517, + 502, + 473, + 505, + 456, + 509, + 486, + 484, + 489, + 496, + 485, + 500, + 521, + 470, + 540, + 482, + 470, + 471, + 504, + 445, + 487, + 470, + 488, + 472, + 470, + 500, + 511, + 484, + 458, + 489, + 503, + 484, + 543, + 502, + 481, + 486, + 494, + 514, + 473, + 491, + 497, + 502, + 482, + 482, + 472, + 512, + 503, + 494, + 531, + 517, + 470, + 534, + 491, + 476, + 482, + 503, + 478, + 469, + 495, + 508, + 487, + 490, + 494, + 476, + 493, + 490, + 464, + 539, + 471, + 498, + 489, + 435, + 459, + 464, + 453, + 498, + 523, + 494, + 518, + 512, + 447, + 451, + 492, + 516, + 496, + 484, + 478, + 512, + 484, + 469, + 518, + 505, + 482, + 493, + 434, + 465, + 503, + 499, + 454, + 502, + 489, + 485, + 521, + 507, + 499, + 483, + 475, + 507, + 504, + 529, + 502, + 500, + 463, + 514, + 517, + 495, + 448, + 517, + 496, + 488, + 532, + 464, + 449, + 509, + 454, + 501, + 500, + 500, + 488, + 503, + 451, + 506, + 481, + 445, + 438, + 485, + 500, + 474, + 499, + 487, + 504, + 474, + 512, + 507, + 470, + 495, + 477, + 465, + 474, + 514, + 492, + 508, + 465, + 477, + 478, + 426, + 505, + 485, + 478, + 501, + 477, + 486, + 446, + 461, + 499, + 483, + 501, + 484, + 483, + 502, + 502, + 464, + 500, + 464, + 479, + 459, + 560, + 467, + 481, + 505, + 496, + 498, + 413, + 512, + 451, + 508, + 443, + 485, + 442, + 480, + 552, + 476, + 487, + 546, + 495, + 518, + 450, + 502, + 502, + 492, + 519, + 497, + 485, + 464, + 445, + 496, + 454, + 501, + 486, + 510, + 501, + 534, + 495, + 466, + 466, + 492, + 482, + 489, + 517, + 437, + 497, + 493, + 423, + 481, + 468, + 481, + 506, + 439, + 439, + 450, + 493, + 489, + 524, + 476, + 512, + 472, + 475, + 504, + 521, + 481, + 500, + 522, + 514, + 486, + 495, + 533, + 464, + 513, + 440, + 499, + 469, + 495, + 454, + 479, + 549, + 514, + 481, + 503, + 468, + 477, + 476, + 492, + 465, + 528, + 493, + 499, + 480, + 491, + 459, + 470, + 476, + 476, + 492, + 474, + 517, + 504, + 466, + 463, + 517, + 453, + 471, + 456, + 508, + 482, + 465, + 475, + 548, + 485, + 483, + 482, + 500, + 549, + 444, + 504, + 481, + 441, + 477, + 496, + 515, + 490, + 532, + 490, + 489, + 444, + 485, + 473, + 453, + 498, + 534, + 467, + 486, + 517, + 499, + 488, + 492, + 499, + 461, + 478, + 484, + 515, + 515, + 486, + 497, + 464, + 523, + 483, + 468, + 491, + 513, + 525, + 489, + 476, + 494, + 486, + 543, + 475, + 474, + 470, + 495, + 466, + 515, + 503, + 514, + 520, + 485, + 485, + 469, + 518, + 493, + 486, + 505, + 457, + 484, + 476, + 502, + 511, + 527, + 481, + 510, + 492, + 525, + 502, + 433, + 490, + 471, + 481, + 527, + 502, + 496, + 470, + 513, + 505, + 458, + 477, + 445, + 545, + 453, + 479, + 484, + 483, + 498, + 494, + 499, + 447, + 486, + 482, + 493, + 474, + 483, + 499, + 454, + 485, + 455, + 473, + 522, + 533, + 463, + 463, + 481, + 495, + 491, + 482, + 528, + 485, + 434, + 450, + 473, + 482, + 501, + 492, + 518, + 468, + 498, + 513, + 498, + 472, + 486, + 531, + 487, + 466, + 501, + 459, + 481, + 497, + 491, + 465, + 494, + 493, + 524, + 490, + 508, + 509, + 437, + 489, + 505, + 480, + 488, + 486, + 501, + 460, + 515, + 472, + 470, + 476, + 442, + 480, + 485, + 504, + 477, + 494, + 473, + 475, + 513, + 506, + 457, + 481, + 482, + 472, + 496, + 475, + 490, + 486, + 470, + 497, + 515, + 471, + 446, + 463, + 518, + 490, + 461, + 509, + 477, + 441, + 486, + 475, + 444, + 501, + 486, + 471, + 513, + 496, + 484, + 511, + 498, + 487, + 478, + 496, + 485, + 443, + 469, + 483, + 460, + 484, + 492, + 493, + 487, + 483, + 527, + 459, + 437, + 496, + 492, + 475, + 457, + 503, + 469, + 509, + 481, + 489, + 465, + 472, + 472, + 488, + 482, + 480, + 512, + 466, + 456, + 458, + 490, + 498, + 488, + 500, + 447, + 496, + 513, + 482, + 494, + 488, + 501, + 495, + 466, + 483, + 491, + 475, + 490, + 505, + 527, + 448, + 501, + 515, + 416, + 493, + 491, + 481, + 475, + 484, + 445, + 486, + 504, + 499, + 545, + 488, + 497, + 469, + 479, + 456, + 508, + 477, + 481, + 538, + 503, + 498, + 444, + 508, + 484, + 490, + 484, + 497, + 509, + 458, + 497, + 458, + 488, + 494, + 456, + 491, + 491, + 489, + 490, + 505, + 520, + 511, + 490, + 458, + 477, + 471, + 481, + 492, + 482, + 504, + 526, + 503, + 507, + 483, + 491, + 504, + 487, + 499, + 551, + 476, + 513, + 467, + 483, + 497, + 529, + 521, + 455, + 473, + 500, + 415, + 474, + 559, + 518, + 512, + 483, + 502, + 462, + 547, + 514, + 503, + 476, + 491, + 488, + 471, + 488, + 451, + 509, + 486, + 503, + 483, + 450, + 461, + 486, + 478, + 508, + 501, + 520, + 513, + 499, + 493, + 467, + 494, + 492, + 506, + 481, + 474, + 498, + 499, + 508, + 481, + 503, + 490, + 484, + 472, + 486, + 505, + 469, + 506, + 499, + 489, + 454, + 487, + 526, + 465, + 458, + 504, + 464, + 517, + 490, + 465, + 503, + 488, + 482, + 495, + 526, + 489, + 477, + 507, + 494, + 525, + 505, + 496, + 491, + 536, + 469, + 506, + 438, + 482, + 517, + 464, + 472, + 507, + 499, + 447, + 465, + 460, + 466, + 484, + 476, + 467, + 484, + 474, + 462, + 516, + 471, + 500, + 511, + 517, + 460, + 512, + 484, + 461, + 458, + 519, + 481, + 505, + 472, + 475, + 488, + 543, + 539, + 511, + 489, + 450, + 516, + 492, + 500, + 452, + 500, + 521, + 519, + 470, + 461, + 467, + 458, + 501, + 496, + 482, + 456, + 500, + 496, + 504, + 481, + 499, + 480, + 477, + 559, + 499, + 522, + 491, + 482, + 480, + 493, + 513, + 485, + 463, + 460, + 501, + 507, + 480, + 503, + 511, + 484, + 461, + 443, + 477, + 464, + 486, + 523, + 493, + 465, + 489, + 510, + 467, + 471, + 498, + 482, + 450, + 483, + 498, + 520, + 484, + 509, + 512, + 454, + 461, + 497, + 465, + 466, + 510, + 451, + 496, + 489, + 478, + 516, + 529, + 492, + 490, + 476, + 505, + 487, + 523, + 489, + 495, + 509, + 475, + 501, + 472, + 479, + 501, + 493, + 475, + 510, + 511, + 462, + 472, + 487, + 487, + 516, + 474, + 512, + 498, + 451, + 484, + 493, + 497, + 544, + 461, + 475, + 525, + 487, + 465, + 490, + 493, + 463, + 477, + 525, + 486, + 484, + 494, + 485, + 490, + 521, + 495, + 494, + 518, + 487, + 501, + 468, + 492, + 449, + 461, + 480, + 500, + 497, + 470, + 479, + 521, + 490, + 520, + 508, + 508, + 495, + 481, + 513, + 502, + 516, + 480, + 471, + 483, + 496, + 487, + 497, + 475, + 492, + 513, + 491, + 550, + 487, + 514, + 510, + 513, + 494, + 483, + 479, + 518, + 468, + 485, + 484, + 469, + 494, + 495, + 527, + 487, + 503, + 486, + 468, + 508, + 528, + 478, + 480, + 463, + 508, + 491, + 454, + 472, + 458, + 500, + 488, + 469, + 500, + 489, + 483, + 476, + 496, + 454, + 466, + 486, + 501, + 470, + 509, + 482, + 486, + 494, + 490, + 463, + 463, + 496, + 525, + 489, + 476, + 490, + 513, + 491, + 500, + 545, + 481, + 492, + 500, + 501, + 483, + 467, + 515, + 499, + 457, + 494, + 478, + 495, + 504, + 472, + 494, + 504, + 532, + 456, + 458, + 480, + 477, + 504, + 484, + 494, + 473, + 488, + 466, + 450, + 475, + 484, + 481, + 478, + 500, + 492, + 466, + 505, + 480, + 497, + 462, + 486, + 480, + 463, + 494, + 437, + 477, + 468, + 469, + 510, + 502, + 509, + 499, + 465, + 530, + 511, + 485, + 465, + 489, + 490, + 490, + 490, + 491, + 509, + 514, + 481, + 481, + 498, + 491, + 493, + 514, + 453, + 505, + 480, + 492, + 471, + 498, + 476, + 496, + 461, + 520, + 469, + 467, + 493, + 453, + 495, + 489, + 507, + 442, + 495, + 468, + 462, + 512, + 505, + 477, + 466, + 453, + 475, + 484, + 498, + 509, + 537, + 479, + 480, + 478, + 470, + 465, + 495, + 464, + 485, + 447, + 508, + 478, + 491, + 494, + 471, + 477, + 514, + 453, + 454, + 517, + 526, + 489, + 491, + 457, + 519, + 483, + 522, + 479, + 522, + 514, + 491, + 491, + 494, + 482, + 463, + 503, + 491, + 471, + 508, + 476, + 472, + 500, + 514, + 500, + 458, + 500, + 474, + 497, + 497, + 514, + 526, + 501, + 493, + 519, + 516, + 474, + 478, + 524, + 467, + 473, + 500, + 486, + 490, + 491, + 487, + 468, + 501, + 540, + 493, + 482, + 492, + 502, + 474, + 486, + 512, + 493, + 483, + 457, + 487, + 486, + 502, + 460, + 459, + 466, + 495, + 469, + 525, + 518, + 518, + 517, + 469, + 463, + 459, + 511, + 495, + 490, + 452, + 511, + 486, + 515, + 482, + 464, + 487, + 513, + 479, + 484, + 468, + 499, + 482, + 496, + 494, + 505, + 475, + 470, + 471, + 497, + 464, + 520, + 506, + 518, + 477, + 498, + 465, + 462, + 478, + 496, + 495, + 490, + 486, + 516, + 503, + 483, + 489, + 508, + 481, + 479, + 533, + 491, + 477, + 482, + 514, + 485, + 488, + 531, + 522, + 481, + 480, + 498, + 485, + 490, + 489, + 510, + 512, + 497, + 517, + 499, + 465, + 453, + 467, + 474, + 507, + 496, + 485, + 497, + 492, + 488, + 505, + 464, + 458, + 509, + 479, + 466, + 490, + 535, + 484, + 497, + 469, + 493, + 498, + 518, + 483, + 449, + 480, + 473, + 523, + 504, + 441, + 503, + 451, + 509, + 497, + 509, + 497, + 511, + 470, + 476, + 499, + 517, + 511, + 463, + 504, + 498, + 476, + 480, + 501, + 513, + 507, + 516, + 461, + 486, + 482, + 504, + 466, + 498, + 457, + 487, + 462, + 475, + 471, + 481, + 493, + 475, + 483, + 473, + 473, + 472, + 508, + 453, + 480, + 459, + 491, + 477, + 504, + 497, + 503, + 504, + 488, + 448, + 528, + 509, + 506, + 477, + 502, + 492, + 519, + 488, + 503, + 503, + 510, + 472, + 498, + 503, + 511, + 512, + 481, + 502, + 464, + 510, + 524, + 435, + 499, + 549, + 463, + 479, + 498, + 438, + 492, + 482, + 512, + 497, + 498, + 494, + 494, + 468, + 508, + 510, + 513, + 456, + 476, + 485, + 517, + 479, + 501, + 466, + 492, + 530, + 470, + 493, + 488, + 498, + 506, + 478, + 478, + 476, + 489, + 496, + 490, + 494, + 493, + 478, + 503, + 482, + 475, + 475, + 444, + 482, + 502, + 476, + 463, + 490, + 455, + 486, + 503, + 504, + 476, + 514, + 514, + 503, + 469, + 463, + 512, + 460, + 467, + 461, + 495, + 469, + 471, + 528, + 458, + 511, + 463, + 514, + 472, + 462, + 469, + 458, + 477, + 476, + 492, + 507, + 510, + 425, + 484, + 457, + 475, + 483, + 498, + 528, + 498, + 503, + 501, + 475, + 451, + 525, + 495, + 482, + 454, + 487, + 481, + 462, + 458, + 473, + 451, + 482, + 483, + 486, + 464, + 522, + 483, + 491, + 493, + 503, + 488, + 505, + 525, + 460, + 520, + 488, + 502, + 525, + 492, + 485, + 454, + 498, + 501, + 457, + 535, + 474, + 527, + 484, + 504, + 476, + 491, + 478, + 494, + 502, + 507, + 484, + 472, + 517, + 457, + 479, + 510, + 529, + 485, + 481, + 482, + 501, + 498, + 516, + 462, + 478, + 505, + 516, + 472, + 511, + 486, + 489, + 485, + 493, + 538, + 486, + 496, + 499, + 483, + 513, + 501, + 465, + 500, + 508, + 481, + 485, + 494, + 481, + 488, + 495, + 506, + 456, + 541, + 473, + 488, + 449, + 528, + 481, + 488, + 457, + 476, + 495, + 461, + 517, + 522, + 452, + 472, + 555, + 502, + 497, + 505, + 512, + 506, + 474, + 463, + 463, + 500, + 452, + 488, + 465, + 475, + 510, + 492, + 454, + 514, + 502, + 482, + 473, + 512, + 454, + 492, + 512, + 468, + 483, + 492, + 465, + 466, + 504, + 514, + 478, + 483, + 473, + 509, + 494, + 508, + 521, + 448, + 472, + 461, + 470, + 452, + 504, + 501, + 489, + 495, + 498, + 490, + 525, + 494, + 490, + 532, + 490, + 506, + 540, + 496, + 489, + 462, + 499, + 496, + 510, + 489, + 461, + 486, + 477, + 474, + 492, + 464, + 498, + 479, + 506, + 472, + 501, + 466, + 480, + 454, + 462, + 478, + 542, + 449, + 472, + 506, + 491, + 509, + 459, + 488, + 511, + 479, + 540, + 475, + 520, + 506, + 515, + 493, + 475, + 464, + 491, + 508, + 462, + 475, + 502, + 498, + 480, + 458, + 482, + 505, + 471, + 501, + 498, + 474, + 496, + 473, + 477, + 515, + 533, + 483, + 482, + 484, + 506, + 519, + 514, + 480, + 483, + 493, + 508, + 492, + 468, + 483, + 468, + 496, + 469, + 489, + 524, + 526, + 477, + 484, + 456, + 479, + 464, + 477, + 479, + 511, + 481, + 471, + 481, + 483, + 498, + 488, + 462, + 489, + 455, + 480, + 500, + 443, + 438, + 465, + 465, + 495, + 452, + 521, + 467, + 462, + 483, + 494, + 521, + 464, + 461, + 463, + 496, + 476, + 487, + 463, + 479, + 489, + 449, + 473, + 480, + 478, + 463, + 473, + 493, + 492, + 500, + 469, + 493, + 486, + 489, + 515, + 506, + 454, + 508, + 481, + 465, + 499, + 477, + 522, + 461, + 483, + 473, + 449, + 495, + 516, + 488, + 470, + 503, + 489, + 500, + 469, + 509, + 482, + 500, + 509, + 512, + 485, + 486, + 523, + 486, + 493, + 503, + 495, + 493, + 451, + 491, + 529, + 504, + 472, + 481, + 493, + 466, + 472, + 496, + 497, + 494, + 489, + 490, + 512, + 495, + 482, + 491, + 495, + 448, + 495, + 454, + 483, + 482, + 482, + 506, + 525, + 494, + 497, + 530, + 460, + 506, + 539, + 505, + 488, + 480, + 474, + 480, + 470, + 486, + 486, + 503, + 496, + 489, + 469, + 462, + 457, + 500, + 449, + 442, + 476, + 448, + 508, + 498, + 507, + 465, + 481, + 476, + 465, + 476, + 517, + 477, + 453, + 490, + 502, + 478, + 479, + 454, + 459, + 482, + 492, + 506, + 441, + 492, + 481, + 482, + 478, + 449, + 478, + 503, + 505, + 474, + 524, + 483, + 503, + 492, + 496, + 479, + 496, + 464, + 492, + 506, + 490, + 494, + 483, + 500, + 531, + 486, + 469, + 517, + 502, + 488, + 487, + 497, + 433, + 462, + 516, + 493, + 474, + 500, + 516, + 497, + 475, + 497, + 487, + 481, + 470, + 469, + 499, + 450, + 501, + 494, + 538, + 490, + 511, + 491, + 504, + 481, + 468, + 499, + 473, + 479, + 471, + 499, + 494, + 488, + 475, + 493, + 506, + 478, + 478, + 502, + 461, + 474, + 490, + 520, + 476, + 473, + 451, + 485, + 454, + 475, + 497, + 509, + 474, + 516, + 473, + 464, + 497, + 497, + 458, + 458, + 481, + 453, + 514, + 521, + 513, + 511, + 490, + 486, + 507, + 498, + 504, + 509, + 485, + 502, + 491, + 481, + 478, + 480, + 448, + 499, + 502, + 499, + 522 + ] + }, + { + "counters": [ + 482, + 510, + 472, + 488, + 524, + 444, + 511, + 511, + 477, + 475, + 538, + 524, + 512, + 483, + 549, + 475, + 515, + 495, + 511, + 461, + 529, + 510, + 501, + 471, + 469, + 488, + 505, + 489, + 474, + 472, + 463, + 499, + 477, + 505, + 501, + 431, + 492, + 522, + 512, + 458, + 456, + 534, + 493, + 501, + 512, + 549, + 504, + 491, + 482, + 492, + 451, + 465, + 482, + 477, + 477, + 496, + 484, + 475, + 470, + 472, + 460, + 504, + 461, + 495, + 518, + 501, + 514, + 480, + 465, + 517, + 491, + 479, + 495, + 443, + 487, + 508, + 462, + 467, + 483, + 452, + 473, + 479, + 532, + 497, + 491, + 501, + 464, + 497, + 493, + 483, + 485, + 471, + 532, + 492, + 453, + 505, + 470, + 502, + 499, + 506, + 464, + 465, + 516, + 506, + 479, + 460, + 500, + 493, + 542, + 468, + 485, + 480, + 469, + 468, + 537, + 477, + 467, + 502, + 448, + 499, + 491, + 497, + 467, + 510, + 450, + 487, + 493, + 457, + 488, + 469, + 462, + 488, + 485, + 538, + 518, + 496, + 498, + 478, + 499, + 511, + 493, + 504, + 463, + 489, + 463, + 467, + 491, + 498, + 521, + 482, + 454, + 458, + 517, + 500, + 447, + 455, + 467, + 493, + 481, + 457, + 482, + 469, + 529, + 510, + 481, + 486, + 435, + 498, + 545, + 489, + 482, + 450, + 484, + 496, + 501, + 465, + 471, + 476, + 493, + 499, + 486, + 503, + 458, + 502, + 482, + 479, + 477, + 505, + 455, + 493, + 468, + 438, + 461, + 480, + 458, + 484, + 497, + 494, + 506, + 460, + 471, + 498, + 532, + 470, + 507, + 478, + 456, + 476, + 471, + 512, + 487, + 490, + 535, + 494, + 518, + 515, + 478, + 496, + 515, + 478, + 477, + 475, + 481, + 489, + 516, + 455, + 512, + 508, + 503, + 503, + 483, + 530, + 434, + 475, + 500, + 467, + 512, + 533, + 530, + 507, + 471, + 509, + 459, + 494, + 501, + 478, + 475, + 486, + 458, + 498, + 452, + 469, + 452, + 533, + 473, + 496, + 492, + 476, + 478, + 484, + 474, + 506, + 464, + 509, + 486, + 481, + 488, + 486, + 486, + 486, + 498, + 490, + 485, + 467, + 477, + 513, + 500, + 492, + 489, + 448, + 499, + 479, + 519, + 479, + 477, + 499, + 503, + 497, + 468, + 496, + 513, + 459, + 476, + 477, + 493, + 478, + 496, + 494, + 478, + 507, + 489, + 500, + 448, + 507, + 507, + 506, + 499, + 504, + 507, + 511, + 445, + 498, + 463, + 491, + 500, + 485, + 513, + 511, + 517, + 458, + 484, + 510, + 481, + 517, + 501, + 524, + 509, + 482, + 465, + 482, + 506, + 482, + 474, + 528, + 493, + 463, + 465, + 487, + 459, + 463, + 527, + 533, + 490, + 510, + 472, + 507, + 480, + 494, + 454, + 471, + 516, + 450, + 507, + 475, + 469, + 454, + 486, + 505, + 492, + 535, + 516, + 542, + 478, + 448, + 465, + 471, + 475, + 522, + 505, + 475, + 488, + 469, + 468, + 465, + 487, + 511, + 470, + 525, + 520, + 498, + 463, + 505, + 498, + 491, + 476, + 493, + 503, + 512, + 483, + 512, + 477, + 492, + 493, + 462, + 515, + 471, + 469, + 484, + 523, + 504, + 507, + 466, + 515, + 486, + 464, + 478, + 478, + 497, + 471, + 486, + 472, + 513, + 498, + 465, + 477, + 504, + 493, + 481, + 507, + 523, + 497, + 456, + 493, + 486, + 504, + 481, + 470, + 492, + 513, + 490, + 489, + 481, + 510, + 477, + 496, + 516, + 468, + 512, + 452, + 477, + 479, + 463, + 459, + 469, + 461, + 465, + 480, + 489, + 515, + 485, + 500, + 477, + 505, + 452, + 509, + 526, + 511, + 513, + 471, + 452, + 492, + 501, + 514, + 442, + 520, + 506, + 486, + 495, + 525, + 461, + 462, + 452, + 493, + 479, + 492, + 534, + 501, + 501, + 496, + 461, + 469, + 502, + 502, + 457, + 455, + 451, + 449, + 503, + 475, + 473, + 531, + 437, + 464, + 460, + 511, + 464, + 478, + 493, + 490, + 539, + 488, + 478, + 481, + 510, + 468, + 464, + 480, + 538, + 507, + 502, + 479, + 498, + 478, + 452, + 523, + 517, + 509, + 483, + 478, + 466, + 489, + 504, + 506, + 505, + 495, + 475, + 476, + 520, + 489, + 504, + 496, + 501, + 508, + 480, + 511, + 460, + 491, + 475, + 479, + 491, + 510, + 537, + 509, + 491, + 500, + 477, + 466, + 481, + 463, + 479, + 472, + 478, + 509, + 499, + 458, + 520, + 482, + 506, + 505, + 486, + 469, + 501, + 479, + 476, + 466, + 450, + 474, + 462, + 479, + 500, + 487, + 472, + 462, + 473, + 481, + 492, + 489, + 464, + 465, + 478, + 498, + 505, + 501, + 493, + 502, + 483, + 487, + 490, + 503, + 501, + 501, + 495, + 470, + 494, + 528, + 483, + 477, + 518, + 455, + 482, + 487, + 448, + 514, + 491, + 495, + 484, + 448, + 507, + 488, + 529, + 493, + 466, + 493, + 480, + 481, + 495, + 448, + 501, + 504, + 492, + 479, + 488, + 523, + 515, + 497, + 469, + 497, + 494, + 486, + 500, + 467, + 539, + 503, + 479, + 504, + 504, + 482, + 481, + 494, + 503, + 482, + 527, + 470, + 478, + 505, + 506, + 476, + 487, + 496, + 498, + 499, + 478, + 513, + 491, + 463, + 504, + 467, + 510, + 518, + 512, + 495, + 484, + 504, + 473, + 480, + 478, + 476, + 483, + 490, + 475, + 468, + 457, + 499, + 492, + 464, + 505, + 492, + 488, + 492, + 502, + 512, + 492, + 482, + 528, + 493, + 488, + 491, + 506, + 458, + 516, + 455, + 468, + 510, + 473, + 515, + 481, + 496, + 519, + 465, + 487, + 457, + 517, + 452, + 483, + 478, + 505, + 501, + 511, + 487, + 492, + 488, + 477, + 500, + 481, + 501, + 505, + 466, + 448, + 477, + 470, + 531, + 496, + 494, + 466, + 467, + 465, + 527, + 499, + 439, + 481, + 484, + 472, + 492, + 514, + 508, + 484, + 508, + 480, + 477, + 477, + 460, + 470, + 474, + 495, + 450, + 463, + 520, + 490, + 533, + 483, + 456, + 504, + 491, + 480, + 445, + 486, + 488, + 498, + 473, + 471, + 489, + 462, + 516, + 505, + 478, + 456, + 463, + 509, + 541, + 478, + 465, + 491, + 465, + 522, + 474, + 523, + 516, + 459, + 472, + 491, + 480, + 515, + 521, + 459, + 480, + 528, + 484, + 447, + 443, + 477, + 482, + 463, + 532, + 462, + 498, + 496, + 468, + 480, + 457, + 478, + 524, + 485, + 445, + 465, + 515, + 488, + 504, + 475, + 460, + 481, + 500, + 470, + 493, + 494, + 504, + 483, + 495, + 455, + 502, + 517, + 498, + 472, + 498, + 491, + 496, + 512, + 535, + 515, + 500, + 499, + 468, + 487, + 494, + 486, + 517, + 465, + 470, + 477, + 474, + 557, + 489, + 503, + 478, + 498, + 508, + 485, + 474, + 487, + 466, + 494, + 482, + 487, + 486, + 468, + 521, + 498, + 509, + 507, + 514, + 508, + 497, + 499, + 498, + 487, + 458, + 462, + 496, + 530, + 452, + 488, + 499, + 470, + 507, + 487, + 468, + 455, + 506, + 490, + 465, + 449, + 468, + 498, + 484, + 440, + 499, + 506, + 457, + 492, + 496, + 499, + 452, + 480, + 496, + 484, + 510, + 445, + 471, + 496, + 488, + 501, + 490, + 468, + 500, + 469, + 479, + 478, + 447, + 501, + 502, + 508, + 484, + 508, + 506, + 516, + 500, + 473, + 490, + 482, + 472, + 492, + 510, + 509, + 465, + 506, + 493, + 483, + 506, + 500, + 480, + 495, + 489, + 460, + 471, + 517, + 500, + 487, + 477, + 490, + 483, + 501, + 487, + 502, + 491, + 501, + 457, + 497, + 484, + 495, + 505, + 482, + 436, + 459, + 489, + 508, + 465, + 465, + 472, + 491, + 498, + 457, + 486, + 502, + 457, + 488, + 476, + 469, + 497, + 496, + 476, + 482, + 467, + 528, + 514, + 479, + 466, + 445, + 484, + 477, + 507, + 446, + 449, + 485, + 498, + 517, + 518, + 468, + 472, + 453, + 519, + 508, + 495, + 487, + 493, + 530, + 484, + 492, + 481, + 508, + 485, + 454, + 477, + 492, + 487, + 467, + 451, + 474, + 491, + 496, + 496, + 524, + 472, + 494, + 487, + 495, + 467, + 516, + 480, + 488, + 472, + 474, + 502, + 515, + 447, + 475, + 472, + 498, + 506, + 492, + 471, + 520, + 478, + 465, + 462, + 524, + 514, + 496, + 518, + 501, + 481, + 468, + 509, + 475, + 512, + 498, + 502, + 501, + 466, + 534, + 479, + 501, + 535, + 462, + 452, + 517, + 496, + 472, + 500, + 466, + 496, + 480, + 510, + 491, + 495, + 445, + 575, + 489, + 476, + 473, + 506, + 493, + 502, + 447, + 530, + 494, + 486, + 484, + 524, + 481, + 467, + 488, + 524, + 511, + 504, + 496, + 462, + 542, + 495, + 533, + 498, + 507, + 466, + 500, + 481, + 487, + 476, + 514, + 480, + 497, + 527, + 497, + 472, + 477, + 490, + 507, + 478, + 492, + 475, + 493, + 464, + 488, + 461, + 510, + 503, + 453, + 511, + 480, + 515, + 513, + 464, + 466, + 472, + 479, + 467, + 485, + 499, + 467, + 505, + 497, + 518, + 486, + 485, + 463, + 494, + 510, + 521, + 437, + 464, + 481, + 485, + 505, + 546, + 481, + 476, + 493, + 488, + 481, + 525, + 504, + 484, + 529, + 514, + 467, + 448, + 485, + 476, + 497, + 428, + 492, + 492, + 474, + 500, + 479, + 526, + 467, + 478, + 479, + 489, + 471, + 486, + 465, + 494, + 455, + 447, + 452, + 475, + 500, + 521, + 516, + 482, + 500, + 463, + 459, + 496, + 501, + 488, + 479, + 463, + 498, + 493, + 495, + 483, + 507, + 510, + 453, + 482, + 494, + 485, + 479, + 485, + 518, + 490, + 506, + 483, + 477, + 496, + 468, + 507, + 514, + 491, + 482, + 481, + 498, + 488, + 459, + 459, + 483, + 492, + 494, + 508, + 521, + 457, + 513, + 471, + 489, + 494, + 462, + 496, + 490, + 457, + 500, + 455, + 484, + 500, + 514, + 509, + 511, + 469, + 480, + 477, + 476, + 493, + 472, + 474, + 468, + 498, + 509, + 475, + 484, + 504, + 502, + 492, + 547, + 489, + 477, + 459, + 477, + 508, + 499, + 472, + 473, + 493, + 509, + 469, + 520, + 481, + 436, + 474, + 530, + 479, + 470, + 504, + 488, + 490, + 484, + 457, + 447, + 486, + 491, + 532, + 488, + 487, + 496, + 490, + 513, + 517, + 498, + 489, + 503, + 491, + 514, + 499, + 494, + 497, + 504, + 454, + 516, + 464, + 483, + 482, + 510, + 435, + 467, + 481, + 465, + 458, + 463, + 473, + 456, + 491, + 487, + 454, + 485, + 478, + 476, + 470, + 474, + 473, + 462, + 492, + 505, + 519, + 536, + 488, + 496, + 498, + 466, + 509, + 485, + 498, + 479, + 518, + 504, + 478, + 498, + 505, + 483, + 485, + 495, + 516, + 462, + 514, + 452, + 492, + 510, + 462, + 457, + 502, + 468, + 499, + 551, + 519, + 505, + 488, + 473, + 529, + 489, + 497, + 486, + 526, + 455, + 485, + 483, + 465, + 533, + 494, + 518, + 485, + 516, + 494, + 477, + 473, + 471, + 496, + 499, + 458, + 473, + 482, + 488, + 459, + 430, + 474, + 457, + 512, + 478, + 489, + 467, + 472, + 495, + 473, + 460, + 512, + 475, + 514, + 450, + 465, + 515, + 527, + 486, + 499, + 503, + 486, + 530, + 466, + 474, + 483, + 511, + 511, + 490, + 489, + 483, + 514, + 476, + 469, + 558, + 501, + 513, + 469, + 484, + 491, + 486, + 494, + 502, + 509, + 485, + 479, + 486, + 472, + 448, + 492, + 536, + 521, + 492, + 483, + 494, + 473, + 506, + 496, + 538, + 439, + 540, + 547, + 503, + 497, + 496, + 486, + 491, + 477, + 469, + 524, + 462, + 478, + 492, + 470, + 502, + 491, + 464, + 486, + 524, + 486, + 438, + 520, + 439, + 465, + 470, + 525, + 500, + 460, + 508, + 518, + 508, + 482, + 486, + 481, + 470, + 489, + 545, + 467, + 463, + 517, + 511, + 508, + 475, + 474, + 487, + 468, + 470, + 484, + 464, + 504, + 481, + 531, + 490, + 458, + 513, + 537, + 530, + 502, + 481, + 464, + 502, + 463, + 490, + 517, + 477, + 497, + 490, + 528, + 494, + 482, + 482, + 476, + 479, + 476, + 472, + 477, + 470, + 455, + 460, + 453, + 487, + 469, + 452, + 478, + 463, + 471, + 486, + 504, + 448, + 460, + 514, + 507, + 512, + 485, + 487, + 459, + 502, + 520, + 477, + 474, + 465, + 507, + 472, + 463, + 461, + 494, + 505, + 513, + 508, + 501, + 488, + 487, + 446, + 468, + 461, + 497, + 492, + 482, + 495, + 474, + 471, + 485, + 506, + 477, + 534, + 488, + 502, + 498, + 472, + 493, + 465, + 487, + 487, + 468, + 496, + 510, + 500, + 494, + 527, + 463, + 497, + 504, + 477, + 473, + 537, + 499, + 531, + 498, + 481, + 499, + 479, + 500, + 500, + 503, + 492, + 470, + 518, + 494, + 507, + 522, + 468, + 509, + 466, + 491, + 470, + 482, + 484, + 471, + 492, + 484, + 492, + 489, + 489, + 462, + 453, + 491, + 493, + 474, + 450, + 486, + 519, + 481, + 504, + 498, + 525, + 502, + 471, + 458, + 463, + 513, + 537, + 492, + 487, + 521, + 495, + 459, + 483, + 478, + 491, + 479, + 466, + 482, + 502, + 455, + 515, + 524, + 479, + 493, + 528, + 481, + 476, + 486, + 507, + 485, + 505, + 490, + 497, + 441, + 482, + 532, + 477, + 456, + 438, + 514, + 456, + 495, + 477, + 545, + 507, + 458, + 478, + 484, + 497, + 503, + 468, + 493, + 489, + 446, + 490, + 491, + 471, + 494, + 485, + 506, + 478, + 498, + 461, + 479, + 474, + 461, + 473, + 509, + 501, + 491, + 434, + 462, + 519, + 491, + 504, + 506, + 510, + 491, + 475, + 512, + 482, + 506, + 515, + 518, + 461, + 478, + 466, + 527, + 520, + 504, + 509, + 472, + 465, + 439, + 463, + 470, + 522, + 512, + 486, + 483, + 473, + 492, + 517, + 521, + 457, + 463, + 495, + 501, + 518, + 484, + 473, + 496, + 484, + 461, + 515, + 487, + 478, + 508, + 487, + 456, + 474, + 474, + 502, + 502, + 485, + 488, + 500, + 493, + 459, + 537, + 482, + 442, + 478, + 458, + 494, + 460, + 532, + 478, + 436, + 528, + 495, + 507, + 471, + 495, + 482, + 505, + 487, + 484, + 488, + 506, + 536, + 494, + 481, + 486, + 490, + 529, + 511, + 473, + 477, + 483, + 504, + 492, + 517, + 501, + 513, + 473, + 471, + 487, + 490, + 504, + 511, + 511, + 467, + 442, + 500, + 491, + 506, + 513, + 496, + 491, + 514, + 490, + 490, + 509, + 488, + 509, + 464, + 505, + 448, + 477, + 476, + 480, + 488, + 457, + 503, + 515, + 483, + 462, + 464, + 477, + 499, + 473, + 492, + 489, + 465, + 541, + 483, + 520, + 519, + 458, + 479, + 474, + 451, + 492, + 458, + 493, + 518, + 511, + 512, + 494, + 479, + 491, + 435, + 483, + 457, + 511, + 523, + 505, + 476, + 513, + 480, + 525, + 507, + 498, + 485, + 488, + 462, + 488, + 494, + 499, + 479, + 472, + 517, + 480, + 503, + 469, + 458, + 457, + 483, + 480, + 464, + 496, + 506, + 461, + 474, + 486, + 487, + 486, + 483, + 508, + 471, + 518, + 454, + 499, + 468, + 482, + 476, + 473, + 507, + 524, + 477, + 474, + 557, + 470, + 443, + 543, + 519, + 482, + 492, + 483, + 498, + 461, + 503, + 516, + 496, + 485, + 487, + 474, + 496, + 541, + 491, + 461, + 507, + 530, + 446, + 499, + 488, + 523, + 499, + 490, + 473, + 488, + 493, + 478, + 490, + 476, + 508, + 470, + 465, + 503, + 484, + 495, + 520, + 483, + 531, + 447, + 502, + 481, + 483, + 463, + 498, + 481, + 498, + 522, + 488, + 473, + 510, + 507, + 489, + 453, + 470, + 469, + 494, + 476, + 516, + 463, + 484, + 433, + 505, + 480, + 517, + 474, + 513, + 481, + 528, + 474, + 486, + 515, + 488, + 485, + 514, + 457, + 480, + 519, + 486, + 430, + 513, + 488, + 471, + 474, + 481, + 490, + 496, + 459, + 453, + 499, + 477, + 474, + 503, + 496, + 470, + 479, + 471, + 449, + 476, + 482, + 495, + 481, + 483, + 493, + 470, + 521, + 506, + 495, + 485, + 463, + 491, + 471, + 500, + 447, + 464, + 520, + 501, + 477, + 517, + 492, + 471, + 486, + 459, + 477, + 495, + 471, + 504, + 455, + 448, + 532, + 498, + 494, + 513, + 462, + 457, + 497, + 507, + 508, + 498, + 488, + 484, + 525, + 444, + 493, + 498, + 492, + 501, + 506, + 478 + ] + }, + { + "counters": [ + 495, + 433, + 458, + 465, + 476, + 491, + 493, + 463, + 501, + 466, + 487, + 490, + 482, + 435, + 458, + 487, + 483, + 500, + 521, + 516, + 476, + 527, + 486, + 452, + 482, + 513, + 478, + 500, + 491, + 473, + 496, + 464, + 489, + 498, + 479, + 526, + 511, + 547, + 489, + 504, + 457, + 522, + 501, + 513, + 463, + 516, + 466, + 462, + 500, + 505, + 533, + 473, + 444, + 508, + 504, + 495, + 488, + 504, + 488, + 502, + 509, + 447, + 473, + 494, + 516, + 481, + 465, + 497, + 475, + 517, + 451, + 477, + 470, + 483, + 446, + 486, + 517, + 495, + 481, + 505, + 487, + 487, + 493, + 527, + 474, + 500, + 458, + 497, + 513, + 521, + 468, + 491, + 496, + 502, + 468, + 492, + 436, + 483, + 491, + 478, + 505, + 516, + 489, + 508, + 500, + 438, + 491, + 471, + 454, + 475, + 507, + 507, + 503, + 449, + 463, + 465, + 462, + 483, + 467, + 482, + 473, + 471, + 526, + 461, + 461, + 488, + 460, + 468, + 449, + 483, + 485, + 471, + 476, + 490, + 508, + 486, + 489, + 492, + 526, + 481, + 452, + 493, + 463, + 472, + 480, + 492, + 537, + 466, + 537, + 478, + 489, + 468, + 512, + 480, + 512, + 479, + 473, + 477, + 472, + 471, + 468, + 477, + 475, + 486, + 490, + 506, + 500, + 490, + 508, + 496, + 455, + 479, + 507, + 449, + 521, + 474, + 507, + 484, + 531, + 485, + 488, + 508, + 506, + 450, + 464, + 481, + 493, + 487, + 460, + 450, + 487, + 493, + 452, + 459, + 526, + 492, + 497, + 484, + 487, + 527, + 500, + 517, + 485, + 504, + 485, + 484, + 469, + 530, + 469, + 453, + 493, + 485, + 472, + 531, + 475, + 490, + 480, + 504, + 495, + 508, + 496, + 528, + 458, + 484, + 468, + 536, + 495, + 473, + 455, + 513, + 477, + 485, + 459, + 499, + 479, + 461, + 477, + 517, + 502, + 490, + 460, + 499, + 475, + 484, + 505, + 502, + 470, + 482, + 530, + 503, + 478, + 481, + 488, + 490, + 493, + 490, + 481, + 492, + 436, + 485, + 505, + 489, + 513, + 518, + 531, + 474, + 479, + 476, + 495, + 460, + 516, + 473, + 470, + 505, + 467, + 472, + 567, + 474, + 482, + 513, + 457, + 502, + 522, + 476, + 483, + 471, + 506, + 458, + 467, + 472, + 460, + 444, + 533, + 509, + 516, + 505, + 495, + 462, + 499, + 507, + 493, + 453, + 458, + 474, + 475, + 476, + 508, + 494, + 469, + 477, + 462, + 468, + 510, + 502, + 495, + 421, + 499, + 459, + 542, + 481, + 481, + 503, + 487, + 458, + 465, + 510, + 482, + 466, + 468, + 468, + 485, + 486, + 478, + 448, + 493, + 524, + 540, + 529, + 470, + 494, + 481, + 478, + 500, + 495, + 487, + 529, + 484, + 508, + 467, + 531, + 477, + 509, + 466, + 473, + 481, + 497, + 500, + 524, + 468, + 505, + 542, + 457, + 493, + 463, + 495, + 515, + 492, + 521, + 496, + 486, + 466, + 478, + 499, + 489, + 464, + 475, + 458, + 526, + 505, + 498, + 499, + 457, + 491, + 497, + 500, + 478, + 494, + 521, + 513, + 492, + 533, + 460, + 507, + 493, + 486, + 502, + 513, + 481, + 512, + 523, + 487, + 498, + 475, + 472, + 536, + 509, + 491, + 449, + 500, + 477, + 516, + 474, + 484, + 488, + 490, + 469, + 493, + 495, + 473, + 454, + 472, + 478, + 497, + 491, + 527, + 491, + 493, + 473, + 475, + 488, + 466, + 468, + 486, + 502, + 454, + 476, + 476, + 495, + 476, + 485, + 445, + 464, + 505, + 501, + 486, + 525, + 538, + 478, + 487, + 495, + 506, + 487, + 484, + 496, + 525, + 491, + 493, + 502, + 452, + 506, + 504, + 491, + 517, + 472, + 466, + 516, + 466, + 486, + 464, + 517, + 516, + 480, + 508, + 493, + 499, + 518, + 515, + 507, + 469, + 517, + 494, + 475, + 457, + 471, + 482, + 490, + 502, + 483, + 492, + 499, + 486, + 487, + 514, + 478, + 507, + 513, + 500, + 512, + 487, + 460, + 519, + 523, + 483, + 469, + 478, + 462, + 492, + 474, + 431, + 478, + 482, + 479, + 510, + 486, + 514, + 480, + 533, + 495, + 446, + 486, + 506, + 482, + 498, + 475, + 487, + 541, + 478, + 464, + 528, + 505, + 485, + 530, + 476, + 489, + 497, + 491, + 493, + 501, + 449, + 522, + 483, + 482, + 463, + 507, + 477, + 443, + 457, + 467, + 477, + 480, + 501, + 486, + 516, + 484, + 460, + 508, + 501, + 493, + 494, + 497, + 460, + 456, + 520, + 492, + 496, + 479, + 458, + 480, + 449, + 501, + 482, + 457, + 483, + 498, + 505, + 535, + 479, + 462, + 535, + 503, + 457, + 510, + 502, + 510, + 492, + 503, + 493, + 492, + 453, + 479, + 486, + 436, + 489, + 494, + 504, + 517, + 484, + 475, + 477, + 436, + 491, + 478, + 477, + 493, + 529, + 482, + 436, + 513, + 513, + 491, + 526, + 481, + 456, + 490, + 469, + 501, + 482, + 480, + 494, + 520, + 502, + 460, + 463, + 488, + 496, + 501, + 462, + 493, + 486, + 523, + 482, + 482, + 478, + 475, + 473, + 507, + 460, + 491, + 461, + 524, + 498, + 479, + 477, + 513, + 492, + 478, + 475, + 510, + 517, + 468, + 488, + 492, + 473, + 468, + 482, + 512, + 483, + 512, + 473, + 468, + 509, + 497, + 480, + 494, + 482, + 551, + 499, + 471, + 472, + 473, + 477, + 508, + 492, + 508, + 441, + 497, + 498, + 481, + 488, + 526, + 483, + 502, + 507, + 484, + 516, + 506, + 473, + 479, + 505, + 538, + 456, + 455, + 479, + 506, + 465, + 456, + 510, + 447, + 498, + 493, + 459, + 503, + 492, + 496, + 524, + 494, + 474, + 482, + 503, + 507, + 496, + 482, + 484, + 499, + 499, + 513, + 442, + 514, + 478, + 475, + 483, + 487, + 484, + 497, + 537, + 478, + 515, + 496, + 494, + 461, + 490, + 494, + 464, + 481, + 465, + 455, + 494, + 506, + 469, + 490, + 470, + 492, + 475, + 491, + 450, + 469, + 458, + 496, + 489, + 491, + 481, + 492, + 471, + 452, + 512, + 480, + 496, + 500, + 516, + 494, + 504, + 496, + 485, + 524, + 461, + 499, + 478, + 516, + 497, + 481, + 488, + 499, + 474, + 498, + 487, + 467, + 483, + 505, + 468, + 498, + 474, + 475, + 509, + 468, + 502, + 471, + 505, + 469, + 477, + 493, + 476, + 477, + 441, + 470, + 479, + 487, + 452, + 540, + 530, + 535, + 527, + 512, + 469, + 520, + 488, + 472, + 461, + 468, + 514, + 505, + 494, + 511, + 492, + 517, + 541, + 488, + 477, + 509, + 454, + 503, + 495, + 518, + 521, + 476, + 506, + 488, + 464, + 479, + 457, + 515, + 470, + 473, + 504, + 521, + 489, + 469, + 460, + 529, + 524, + 499, + 470, + 480, + 494, + 486, + 509, + 510, + 508, + 439, + 475, + 497, + 510, + 474, + 500, + 481, + 477, + 515, + 464, + 498, + 523, + 491, + 492, + 485, + 512, + 481, + 492, + 437, + 474, + 486, + 519, + 470, + 483, + 475, + 489, + 521, + 482, + 482, + 532, + 494, + 534, + 502, + 487, + 496, + 487, + 511, + 507, + 502, + 499, + 506, + 477, + 463, + 493, + 525, + 534, + 484, + 500, + 511, + 518, + 488, + 510, + 428, + 471, + 482, + 461, + 500, + 509, + 462, + 479, + 512, + 471, + 461, + 468, + 469, + 501, + 494, + 470, + 486, + 507, + 511, + 433, + 520, + 535, + 497, + 473, + 450, + 535, + 492, + 471, + 473, + 507, + 429, + 427, + 510, + 458, + 413, + 470, + 478, + 490, + 470, + 496, + 468, + 492, + 483, + 464, + 488, + 532, + 462, + 476, + 516, + 459, + 488, + 497, + 505, + 497, + 457, + 474, + 448, + 489, + 502, + 468, + 509, + 506, + 521, + 466, + 484, + 521, + 476, + 492, + 476, + 481, + 490, + 468, + 497, + 477, + 462, + 467, + 497, + 475, + 521, + 490, + 470, + 471, + 507, + 486, + 481, + 511, + 501, + 499, + 508, + 460, + 478, + 491, + 487, + 520, + 497, + 478, + 472, + 458, + 465, + 496, + 482, + 493, + 492, + 486, + 491, + 527, + 474, + 495, + 511, + 471, + 486, + 485, + 494, + 495, + 478, + 465, + 507, + 479, + 508, + 507, + 500, + 482, + 516, + 498, + 492, + 521, + 524, + 503, + 488, + 493, + 463, + 518, + 537, + 483, + 480, + 465, + 527, + 495, + 493, + 508, + 482, + 495, + 521, + 492, + 494, + 481, + 475, + 443, + 490, + 479, + 477, + 494, + 471, + 484, + 431, + 452, + 436, + 501, + 480, + 477, + 467, + 490, + 484, + 474, + 483, + 508, + 507, + 507, + 530, + 525, + 489, + 497, + 502, + 507, + 485, + 478, + 516, + 472, + 473, + 485, + 477, + 465, + 513, + 463, + 487, + 514, + 489, + 454, + 476, + 480, + 484, + 479, + 513, + 462, + 486, + 520, + 504, + 515, + 458, + 513, + 526, + 491, + 451, + 489, + 520, + 515, + 454, + 463, + 481, + 493, + 498, + 508, + 482, + 474, + 489, + 457, + 461, + 481, + 499, + 482, + 524, + 509, + 511, + 447, + 509, + 482, + 473, + 478, + 507, + 471, + 486, + 532, + 516, + 474, + 489, + 540, + 510, + 514, + 477, + 497, + 463, + 492, + 507, + 515, + 495, + 510, + 469, + 487, + 482, + 472, + 504, + 506, + 503, + 522, + 501, + 491, + 482, + 487, + 530, + 474, + 498, + 478, + 477, + 491, + 495, + 460, + 512, + 474, + 460, + 483, + 518, + 532, + 487, + 483, + 509, + 529, + 490, + 515, + 483, + 487, + 448, + 488, + 494, + 474, + 454, + 495, + 473, + 479, + 472, + 462, + 502, + 439, + 464, + 519, + 456, + 501, + 436, + 487, + 490, + 502, + 468, + 442, + 483, + 504, + 473, + 495, + 528, + 475, + 498, + 483, + 529, + 476, + 489, + 509, + 474, + 457, + 470, + 487, + 480, + 451, + 475, + 459, + 485, + 490, + 493, + 492, + 421, + 501, + 480, + 446, + 482, + 486, + 501, + 497, + 512, + 482, + 456, + 502, + 451, + 489, + 515, + 462, + 494, + 475, + 496, + 502, + 495, + 471, + 490, + 440, + 491, + 489, + 506, + 487, + 490, + 459, + 491, + 459, + 469, + 468, + 445, + 487, + 494, + 491, + 490, + 474, + 454, + 480, + 503, + 476, + 491, + 459, + 444, + 480, + 497, + 423, + 512, + 508, + 528, + 460, + 501, + 495, + 518, + 477, + 506, + 498, + 496, + 507, + 468, + 462, + 462, + 508, + 502, + 468, + 485, + 472, + 475, + 487, + 493, + 522, + 484, + 513, + 513, + 525, + 491, + 513, + 479, + 414, + 494, + 473, + 450, + 494, + 474, + 497, + 525, + 465, + 507, + 505, + 456, + 492, + 458, + 465, + 457, + 534, + 537, + 475, + 481, + 481, + 470, + 500, + 507, + 487, + 469, + 463, + 473, + 509, + 497, + 456, + 485, + 490, + 494, + 486, + 501, + 481, + 529, + 492, + 517, + 488, + 501, + 493, + 486, + 481, + 547, + 509, + 505, + 507, + 480, + 510, + 491, + 524, + 519, + 469, + 488, + 493, + 464, + 467, + 519, + 441, + 493, + 505, + 496, + 481, + 470, + 528, + 467, + 495, + 499, + 490, + 487, + 480, + 537, + 495, + 471, + 469, + 497, + 487, + 494, + 447, + 454, + 453, + 493, + 523, + 473, + 453, + 510, + 499, + 465, + 508, + 509, + 499, + 446, + 493, + 511, + 520, + 484, + 503, + 458, + 447, + 454, + 459, + 474, + 461, + 483, + 507, + 455, + 464, + 498, + 485, + 492, + 472, + 456, + 468, + 525, + 508, + 459, + 493, + 472, + 501, + 503, + 485, + 471, + 510, + 439, + 494, + 494, + 500, + 490, + 468, + 497, + 516, + 467, + 481, + 473, + 474, + 501, + 512, + 487, + 517, + 461, + 495, + 475, + 508, + 480, + 497, + 515, + 530, + 489, + 490, + 454, + 483, + 475, + 492, + 521, + 488, + 495, + 492, + 480, + 524, + 484, + 479, + 457, + 481, + 526, + 502, + 480, + 476, + 452, + 530, + 481, + 518, + 498, + 509, + 439, + 472, + 492, + 490, + 517, + 508, + 502, + 508, + 511, + 488, + 510, + 504, + 491, + 509, + 462, + 457, + 474, + 508, + 523, + 488, + 481, + 539, + 492, + 454, + 523, + 511, + 500, + 493, + 436, + 461, + 450, + 515, + 497, + 462, + 490, + 483, + 419, + 510, + 475, + 461, + 468, + 506, + 492, + 464, + 469, + 501, + 481, + 512, + 473, + 513, + 462, + 500, + 513, + 475, + 462, + 484, + 491, + 470, + 452, + 459, + 516, + 529, + 539, + 479, + 486, + 489, + 508, + 515, + 502, + 465, + 471, + 486, + 479, + 464, + 492, + 481, + 490, + 478, + 458, + 514, + 502, + 506, + 477, + 478, + 505, + 466, + 531, + 474, + 475, + 498, + 517, + 504, + 480, + 537, + 481, + 480, + 471, + 487, + 480, + 466, + 484, + 499, + 465, + 466, + 503, + 420, + 496, + 525, + 512, + 520, + 451, + 495, + 502, + 482, + 484, + 492, + 507, + 495, + 479, + 505, + 489, + 513, + 466, + 464, + 482, + 511, + 467, + 483, + 514, + 483, + 490, + 522, + 478, + 455, + 473, + 511, + 485, + 479, + 499, + 499, + 487, + 499, + 444, + 493, + 491, + 477, + 511, + 456, + 507, + 529, + 506, + 476, + 441, + 508, + 495, + 477, + 494, + 461, + 516, + 525, + 490, + 488, + 496, + 515, + 479, + 582, + 513, + 463, + 496, + 498, + 442, + 477, + 509, + 499, + 473, + 495, + 490, + 525, + 474, + 496, + 458, + 501, + 471, + 474, + 501, + 477, + 503, + 465, + 517, + 520, + 469, + 506, + 492, + 508, + 481, + 488, + 507, + 491, + 531, + 526, + 463, + 517, + 452, + 474, + 488, + 503, + 492, + 505, + 442, + 474, + 461, + 493, + 466, + 493, + 461, + 486, + 475, + 480, + 507, + 502, + 493, + 490, + 490, + 483, + 479, + 484, + 507, + 451, + 465, + 450, + 517, + 502, + 502, + 485, + 481, + 471, + 498, + 465, + 462, + 484, + 437, + 460, + 495, + 438, + 462, + 462, + 510, + 529, + 484, + 485, + 488, + 509, + 453, + 499, + 500, + 499, + 479, + 528, + 487, + 483, + 481, + 474, + 489, + 478, + 475, + 471, + 486, + 476, + 479, + 534, + 480, + 494, + 482, + 507, + 530, + 487, + 497, + 473, + 473, + 472, + 458, + 442, + 440, + 459, + 505, + 502, + 498, + 467, + 500, + 505, + 517, + 522, + 479, + 494, + 487, + 457, + 472, + 480, + 517, + 483, + 514, + 486, + 471, + 457, + 504, + 471, + 466, + 494, + 508, + 465, + 487, + 522, + 500, + 484, + 487, + 482, + 450, + 494, + 527, + 483, + 479, + 472, + 521, + 464, + 486, + 512, + 489, + 503, + 478, + 492, + 527, + 507, + 528, + 507, + 478, + 462, + 461, + 462, + 497, + 543, + 498, + 477, + 492, + 493, + 466, + 480, + 509, + 530, + 448, + 522, + 475, + 484, + 455, + 444, + 477, + 491, + 494, + 450, + 494, + 507, + 515, + 447, + 512, + 506, + 486, + 452, + 491, + 472, + 513, + 511, + 503, + 504, + 477, + 501, + 516, + 494, + 480, + 516, + 482, + 480, + 483, + 440, + 492, + 464, + 501, + 498, + 506, + 500, + 497, + 472, + 527, + 520, + 459, + 504, + 495, + 474, + 489, + 452, + 499, + 485, + 466, + 512, + 500, + 485, + 516, + 495, + 476, + 514, + 494, + 497, + 492, + 476, + 501, + 501, + 465, + 510, + 502, + 476, + 525, + 517, + 526, + 521, + 491, + 481, + 480, + 455, + 469, + 477, + 523, + 515, + 511, + 429, + 481, + 522, + 515, + 473, + 481, + 511, + 492, + 479, + 478, + 479, + 526, + 499, + 438, + 491, + 535, + 497, + 426, + 495, + 489, + 465, + 467, + 485, + 443, + 461, + 466, + 462, + 467, + 553, + 502, + 504, + 496, + 481, + 454, + 454, + 458, + 528, + 493, + 482, + 505, + 495, + 522, + 473, + 494, + 442, + 499, + 484, + 496, + 561, + 438, + 512, + 468, + 514, + 472, + 464, + 503, + 526, + 480, + 512, + 467, + 465, + 444, + 504, + 503, + 486, + 490, + 492, + 490, + 524, + 474, + 491, + 488, + 497, + 514, + 483, + 452, + 472, + 532, + 491, + 481, + 487, + 509, + 512, + 507, + 490, + 456, + 511, + 507, + 477, + 452, + 510, + 469, + 481, + 467, + 497, + 461, + 510, + 487, + 491, + 508, + 467, + 463, + 433, + 500, + 488, + 483, + 460, + 508, + 512, + 447, + 470, + 466, + 492, + 502, + 510, + 481, + 537, + 515, + 524, + 506, + 501, + 535, + 526, + 537, + 492, + 466, + 518, + 502, + 528, + 494, + 416, + 483, + 465, + 505, + 472, + 518 + ] + } + ], + "default_value": 0 + }, + "null_count": 0, + "tot_col_size": 2991746, + "last_update_version": 417600051000901632, + "correlation": -1 + } + }, + "indices": { + "idx_a": { + "histogram": { + "ndv": 1000000, + "buckets": [ + { + "count": 4096, + "lower_bound": "A4AAAAAAAAAB", + "upper_bound": "A4AAAAAAABAA", + "repeats": 1 + }, + { + "count": 8192, + "lower_bound": "A4AAAAAAABAB", + "upper_bound": "A4AAAAAAACAA", + "repeats": 1 + }, + { + "count": 12288, + "lower_bound": "A4AAAAAAACAB", + "upper_bound": "A4AAAAAAADAA", + "repeats": 1 + }, + { + "count": 16384, + "lower_bound": "A4AAAAAAADAB", + "upper_bound": "A4AAAAAAAEAA", + "repeats": 1 + }, + { + "count": 20480, + "lower_bound": "A4AAAAAAAEAB", + "upper_bound": "A4AAAAAAAFAA", + "repeats": 1 + }, + { + "count": 24576, + "lower_bound": "A4AAAAAAAFAB", + "upper_bound": "A4AAAAAAAGAA", + "repeats": 1 + }, + { + "count": 28672, + "lower_bound": "A4AAAAAAAGAB", + "upper_bound": "A4AAAAAAAHAA", + "repeats": 1 + }, + { + "count": 32768, + "lower_bound": "A4AAAAAAAHAB", + "upper_bound": "A4AAAAAAAIAA", + "repeats": 1 + }, + { + "count": 36864, + "lower_bound": "A4AAAAAAAIAB", + "upper_bound": "A4AAAAAAAJAA", + "repeats": 1 + }, + { + "count": 40960, + "lower_bound": "A4AAAAAAAJAB", + "upper_bound": "A4AAAAAAAKAA", + "repeats": 1 + }, + { + "count": 45056, + "lower_bound": "A4AAAAAAAKAB", + "upper_bound": "A4AAAAAAALAA", + "repeats": 1 + }, + { + "count": 49152, + "lower_bound": "A4AAAAAAALAB", + "upper_bound": "A4AAAAAAAMAA", + "repeats": 1 + }, + { + "count": 53248, + "lower_bound": "A4AAAAAAAMAB", + "upper_bound": "A4AAAAAAANAA", + "repeats": 1 + }, + { + "count": 57344, + "lower_bound": "A4AAAAAAANAB", + "upper_bound": "A4AAAAAAAOAA", + "repeats": 1 + }, + { + "count": 61440, + "lower_bound": "A4AAAAAAAOAB", + "upper_bound": "A4AAAAAAAPAA", + "repeats": 1 + }, + { + "count": 65536, + "lower_bound": "A4AAAAAAAPAB", + "upper_bound": "A4AAAAAAAQAA", + "repeats": 1 + }, + { + "count": 69632, + "lower_bound": "A4AAAAAAAQAB", + "upper_bound": "A4AAAAAAARAA", + "repeats": 1 + }, + { + "count": 73728, + "lower_bound": "A4AAAAAAARAB", + "upper_bound": "A4AAAAAAASAA", + "repeats": 1 + }, + { + "count": 77824, + "lower_bound": "A4AAAAAAASAB", + "upper_bound": "A4AAAAAAATAA", + "repeats": 1 + }, + { + "count": 81920, + "lower_bound": "A4AAAAAAATAB", + "upper_bound": "A4AAAAAAAUAA", + "repeats": 1 + }, + { + "count": 86016, + "lower_bound": "A4AAAAAAAUAB", + "upper_bound": "A4AAAAAAAVAA", + "repeats": 1 + }, + { + "count": 90112, + "lower_bound": "A4AAAAAAAVAB", + "upper_bound": "A4AAAAAAAWAA", + "repeats": 1 + }, + { + "count": 94208, + "lower_bound": "A4AAAAAAAWAB", + "upper_bound": "A4AAAAAAAXAA", + "repeats": 1 + }, + { + "count": 98304, + "lower_bound": "A4AAAAAAAXAB", + "upper_bound": "A4AAAAAAAYAA", + "repeats": 1 + }, + { + "count": 102400, + "lower_bound": "A4AAAAAAAYAB", + "upper_bound": "A4AAAAAAAZAA", + "repeats": 1 + }, + { + "count": 106496, + "lower_bound": "A4AAAAAAAZAB", + "upper_bound": "A4AAAAAAAaAA", + "repeats": 1 + }, + { + "count": 110592, + "lower_bound": "A4AAAAAAAaAB", + "upper_bound": "A4AAAAAAAbAA", + "repeats": 1 + }, + { + "count": 114688, + "lower_bound": "A4AAAAAAAbAB", + "upper_bound": "A4AAAAAAAcAA", + "repeats": 1 + }, + { + "count": 118784, + "lower_bound": "A4AAAAAAAcAB", + "upper_bound": "A4AAAAAAAdAA", + "repeats": 1 + }, + { + "count": 122880, + "lower_bound": "A4AAAAAAAdAB", + "upper_bound": "A4AAAAAAAeAA", + "repeats": 1 + }, + { + "count": 126976, + "lower_bound": "A4AAAAAAAeAB", + "upper_bound": "A4AAAAAAAfAA", + "repeats": 1 + }, + { + "count": 131072, + "lower_bound": "A4AAAAAAAfAB", + "upper_bound": "A4AAAAAAAgAA", + "repeats": 1 + }, + { + "count": 135168, + "lower_bound": "A4AAAAAAAgAB", + "upper_bound": "A4AAAAAAAhAA", + "repeats": 1 + }, + { + "count": 139264, + "lower_bound": "A4AAAAAAAhAB", + "upper_bound": "A4AAAAAAAiAA", + "repeats": 1 + }, + { + "count": 143360, + "lower_bound": "A4AAAAAAAiAB", + "upper_bound": "A4AAAAAAAjAA", + "repeats": 1 + }, + { + "count": 147456, + "lower_bound": "A4AAAAAAAjAB", + "upper_bound": "A4AAAAAAAkAA", + "repeats": 1 + }, + { + "count": 151552, + "lower_bound": "A4AAAAAAAkAB", + "upper_bound": "A4AAAAAAAlAA", + "repeats": 1 + }, + { + "count": 155648, + "lower_bound": "A4AAAAAAAlAB", + "upper_bound": "A4AAAAAAAmAA", + "repeats": 1 + }, + { + "count": 159744, + "lower_bound": "A4AAAAAAAmAB", + "upper_bound": "A4AAAAAAAnAA", + "repeats": 1 + }, + { + "count": 163840, + "lower_bound": "A4AAAAAAAnAB", + "upper_bound": "A4AAAAAAAoAA", + "repeats": 1 + }, + { + "count": 167936, + "lower_bound": "A4AAAAAAAoAB", + "upper_bound": "A4AAAAAAApAA", + "repeats": 1 + }, + { + "count": 172032, + "lower_bound": "A4AAAAAAApAB", + "upper_bound": "A4AAAAAAAqAA", + "repeats": 1 + }, + { + "count": 176128, + "lower_bound": "A4AAAAAAAqAB", + "upper_bound": "A4AAAAAAArAA", + "repeats": 1 + }, + { + "count": 180224, + "lower_bound": "A4AAAAAAArAB", + "upper_bound": "A4AAAAAAAsAA", + "repeats": 1 + }, + { + "count": 184320, + "lower_bound": "A4AAAAAAAsAB", + "upper_bound": "A4AAAAAAAtAA", + "repeats": 1 + }, + { + "count": 188416, + "lower_bound": "A4AAAAAAAtAB", + "upper_bound": "A4AAAAAAAuAA", + "repeats": 1 + }, + { + "count": 192512, + "lower_bound": "A4AAAAAAAuAB", + "upper_bound": "A4AAAAAAAvAA", + "repeats": 1 + }, + { + "count": 196608, + "lower_bound": "A4AAAAAAAvAB", + "upper_bound": "A4AAAAAAAwAA", + "repeats": 1 + }, + { + "count": 200704, + "lower_bound": "A4AAAAAAAwAB", + "upper_bound": "A4AAAAAAAxAA", + "repeats": 1 + }, + { + "count": 204800, + "lower_bound": "A4AAAAAAAxAB", + "upper_bound": "A4AAAAAAAyAA", + "repeats": 1 + }, + { + "count": 208896, + "lower_bound": "A4AAAAAAAyAB", + "upper_bound": "A4AAAAAAAzAA", + "repeats": 1 + }, + { + "count": 212992, + "lower_bound": "A4AAAAAAAzAB", + "upper_bound": "A4AAAAAAA0AA", + "repeats": 1 + }, + { + "count": 217088, + "lower_bound": "A4AAAAAAA0AB", + "upper_bound": "A4AAAAAAA1AA", + "repeats": 1 + }, + { + "count": 221184, + "lower_bound": "A4AAAAAAA1AB", + "upper_bound": "A4AAAAAAA2AA", + "repeats": 1 + }, + { + "count": 225280, + "lower_bound": "A4AAAAAAA2AB", + "upper_bound": "A4AAAAAAA3AA", + "repeats": 1 + }, + { + "count": 229376, + "lower_bound": "A4AAAAAAA3AB", + "upper_bound": "A4AAAAAAA4AA", + "repeats": 1 + }, + { + "count": 233472, + "lower_bound": "A4AAAAAAA4AB", + "upper_bound": "A4AAAAAAA5AA", + "repeats": 1 + }, + { + "count": 237568, + "lower_bound": "A4AAAAAAA5AB", + "upper_bound": "A4AAAAAAA6AA", + "repeats": 1 + }, + { + "count": 241664, + "lower_bound": "A4AAAAAAA6AB", + "upper_bound": "A4AAAAAAA7AA", + "repeats": 1 + }, + { + "count": 245760, + "lower_bound": "A4AAAAAAA7AB", + "upper_bound": "A4AAAAAAA8AA", + "repeats": 1 + }, + { + "count": 249856, + "lower_bound": "A4AAAAAAA8AB", + "upper_bound": "A4AAAAAAA9AA", + "repeats": 1 + }, + { + "count": 253952, + "lower_bound": "A4AAAAAAA9AB", + "upper_bound": "A4AAAAAAA+AA", + "repeats": 1 + }, + { + "count": 258048, + "lower_bound": "A4AAAAAAA+AB", + "upper_bound": "A4AAAAAAA/AA", + "repeats": 1 + }, + { + "count": 262144, + "lower_bound": "A4AAAAAAA/AB", + "upper_bound": "A4AAAAAABAAA", + "repeats": 1 + }, + { + "count": 266240, + "lower_bound": "A4AAAAAABAAB", + "upper_bound": "A4AAAAAABBAA", + "repeats": 1 + }, + { + "count": 270336, + "lower_bound": "A4AAAAAABBAB", + "upper_bound": "A4AAAAAABCAA", + "repeats": 1 + }, + { + "count": 274432, + "lower_bound": "A4AAAAAABCAB", + "upper_bound": "A4AAAAAABDAA", + "repeats": 1 + }, + { + "count": 278528, + "lower_bound": "A4AAAAAABDAB", + "upper_bound": "A4AAAAAABEAA", + "repeats": 1 + }, + { + "count": 282624, + "lower_bound": "A4AAAAAABEAB", + "upper_bound": "A4AAAAAABFAA", + "repeats": 1 + }, + { + "count": 286720, + "lower_bound": "A4AAAAAABFAB", + "upper_bound": "A4AAAAAABGAA", + "repeats": 1 + }, + { + "count": 290816, + "lower_bound": "A4AAAAAABGAB", + "upper_bound": "A4AAAAAABHAA", + "repeats": 1 + }, + { + "count": 294912, + "lower_bound": "A4AAAAAABHAB", + "upper_bound": "A4AAAAAABIAA", + "repeats": 1 + }, + { + "count": 299008, + "lower_bound": "A4AAAAAABIAB", + "upper_bound": "A4AAAAAABJAA", + "repeats": 1 + }, + { + "count": 303104, + "lower_bound": "A4AAAAAABJAB", + "upper_bound": "A4AAAAAABKAA", + "repeats": 1 + }, + { + "count": 307200, + "lower_bound": "A4AAAAAABKAB", + "upper_bound": "A4AAAAAABLAA", + "repeats": 1 + }, + { + "count": 311296, + "lower_bound": "A4AAAAAABLAB", + "upper_bound": "A4AAAAAABMAA", + "repeats": 1 + }, + { + "count": 315392, + "lower_bound": "A4AAAAAABMAB", + "upper_bound": "A4AAAAAABNAA", + "repeats": 1 + }, + { + "count": 319488, + "lower_bound": "A4AAAAAABNAB", + "upper_bound": "A4AAAAAABOAA", + "repeats": 1 + }, + { + "count": 323584, + "lower_bound": "A4AAAAAABOAB", + "upper_bound": "A4AAAAAABPAA", + "repeats": 1 + }, + { + "count": 327680, + "lower_bound": "A4AAAAAABPAB", + "upper_bound": "A4AAAAAABQAA", + "repeats": 1 + }, + { + "count": 331776, + "lower_bound": "A4AAAAAABQAB", + "upper_bound": "A4AAAAAABRAA", + "repeats": 1 + }, + { + "count": 335872, + "lower_bound": "A4AAAAAABRAB", + "upper_bound": "A4AAAAAABSAA", + "repeats": 1 + }, + { + "count": 339968, + "lower_bound": "A4AAAAAABSAB", + "upper_bound": "A4AAAAAABTAA", + "repeats": 1 + }, + { + "count": 344064, + "lower_bound": "A4AAAAAABTAB", + "upper_bound": "A4AAAAAABUAA", + "repeats": 1 + }, + { + "count": 348160, + "lower_bound": "A4AAAAAABUAB", + "upper_bound": "A4AAAAAABVAA", + "repeats": 1 + }, + { + "count": 352256, + "lower_bound": "A4AAAAAABVAB", + "upper_bound": "A4AAAAAABWAA", + "repeats": 1 + }, + { + "count": 356352, + "lower_bound": "A4AAAAAABWAB", + "upper_bound": "A4AAAAAABXAA", + "repeats": 1 + }, + { + "count": 360448, + "lower_bound": "A4AAAAAABXAB", + "upper_bound": "A4AAAAAABYAA", + "repeats": 1 + }, + { + "count": 364544, + "lower_bound": "A4AAAAAABYAB", + "upper_bound": "A4AAAAAABZAA", + "repeats": 1 + }, + { + "count": 368640, + "lower_bound": "A4AAAAAABZAB", + "upper_bound": "A4AAAAAABaAA", + "repeats": 1 + }, + { + "count": 372736, + "lower_bound": "A4AAAAAABaAB", + "upper_bound": "A4AAAAAABbAA", + "repeats": 1 + }, + { + "count": 376832, + "lower_bound": "A4AAAAAABbAB", + "upper_bound": "A4AAAAAABcAA", + "repeats": 1 + }, + { + "count": 380928, + "lower_bound": "A4AAAAAABcAB", + "upper_bound": "A4AAAAAABdAA", + "repeats": 1 + }, + { + "count": 385024, + "lower_bound": "A4AAAAAABdAB", + "upper_bound": "A4AAAAAABeAA", + "repeats": 1 + }, + { + "count": 389120, + "lower_bound": "A4AAAAAABeAB", + "upper_bound": "A4AAAAAABfAA", + "repeats": 1 + }, + { + "count": 393216, + "lower_bound": "A4AAAAAABfAB", + "upper_bound": "A4AAAAAABgAA", + "repeats": 1 + }, + { + "count": 397312, + "lower_bound": "A4AAAAAABgAB", + "upper_bound": "A4AAAAAABhAA", + "repeats": 1 + }, + { + "count": 401408, + "lower_bound": "A4AAAAAABhAB", + "upper_bound": "A4AAAAAABiAA", + "repeats": 1 + }, + { + "count": 405504, + "lower_bound": "A4AAAAAABiAB", + "upper_bound": "A4AAAAAABjAA", + "repeats": 1 + }, + { + "count": 409600, + "lower_bound": "A4AAAAAABjAB", + "upper_bound": "A4AAAAAABkAA", + "repeats": 1 + }, + { + "count": 413696, + "lower_bound": "A4AAAAAABkAB", + "upper_bound": "A4AAAAAABlAA", + "repeats": 1 + }, + { + "count": 417792, + "lower_bound": "A4AAAAAABlAB", + "upper_bound": "A4AAAAAABmAA", + "repeats": 1 + }, + { + "count": 421888, + "lower_bound": "A4AAAAAABmAB", + "upper_bound": "A4AAAAAABnAA", + "repeats": 1 + }, + { + "count": 425984, + "lower_bound": "A4AAAAAABnAB", + "upper_bound": "A4AAAAAABoAA", + "repeats": 1 + }, + { + "count": 430080, + "lower_bound": "A4AAAAAABoAB", + "upper_bound": "A4AAAAAABpAA", + "repeats": 1 + }, + { + "count": 434176, + "lower_bound": "A4AAAAAABpAB", + "upper_bound": "A4AAAAAABqAA", + "repeats": 1 + }, + { + "count": 438272, + "lower_bound": "A4AAAAAABqAB", + "upper_bound": "A4AAAAAABrAA", + "repeats": 1 + }, + { + "count": 442368, + "lower_bound": "A4AAAAAABrAB", + "upper_bound": "A4AAAAAABsAA", + "repeats": 1 + }, + { + "count": 446464, + "lower_bound": "A4AAAAAABsAB", + "upper_bound": "A4AAAAAABtAA", + "repeats": 1 + }, + { + "count": 450560, + "lower_bound": "A4AAAAAABtAB", + "upper_bound": "A4AAAAAABuAA", + "repeats": 1 + }, + { + "count": 454656, + "lower_bound": "A4AAAAAABuAB", + "upper_bound": "A4AAAAAABvAA", + "repeats": 1 + }, + { + "count": 458752, + "lower_bound": "A4AAAAAABvAB", + "upper_bound": "A4AAAAAABwAA", + "repeats": 1 + }, + { + "count": 462848, + "lower_bound": "A4AAAAAABwAB", + "upper_bound": "A4AAAAAABxAA", + "repeats": 1 + }, + { + "count": 466944, + "lower_bound": "A4AAAAAABxAB", + "upper_bound": "A4AAAAAAByAA", + "repeats": 1 + }, + { + "count": 471040, + "lower_bound": "A4AAAAAAByAB", + "upper_bound": "A4AAAAAABzAA", + "repeats": 1 + }, + { + "count": 475136, + "lower_bound": "A4AAAAAABzAB", + "upper_bound": "A4AAAAAAB0AA", + "repeats": 1 + }, + { + "count": 479232, + "lower_bound": "A4AAAAAAB0AB", + "upper_bound": "A4AAAAAAB1AA", + "repeats": 1 + }, + { + "count": 483328, + "lower_bound": "A4AAAAAAB1AB", + "upper_bound": "A4AAAAAAB2AA", + "repeats": 1 + }, + { + "count": 487424, + "lower_bound": "A4AAAAAAB2AB", + "upper_bound": "A4AAAAAAB3AA", + "repeats": 1 + }, + { + "count": 491520, + "lower_bound": "A4AAAAAAB3AB", + "upper_bound": "A4AAAAAAB4AA", + "repeats": 1 + }, + { + "count": 495616, + "lower_bound": "A4AAAAAAB4AB", + "upper_bound": "A4AAAAAAB5AA", + "repeats": 1 + }, + { + "count": 499712, + "lower_bound": "A4AAAAAAB5AB", + "upper_bound": "A4AAAAAAB6AA", + "repeats": 1 + }, + { + "count": 503808, + "lower_bound": "A4AAAAAAB6AB", + "upper_bound": "A4AAAAAAB7AA", + "repeats": 1 + }, + { + "count": 507904, + "lower_bound": "A4AAAAAAB7AB", + "upper_bound": "A4AAAAAAB8AA", + "repeats": 1 + }, + { + "count": 512000, + "lower_bound": "A4AAAAAAB8AB", + "upper_bound": "A4AAAAAAB9AA", + "repeats": 1 + }, + { + "count": 516096, + "lower_bound": "A4AAAAAAB9AB", + "upper_bound": "A4AAAAAAB+AA", + "repeats": 1 + }, + { + "count": 520192, + "lower_bound": "A4AAAAAAB+AB", + "upper_bound": "A4AAAAAAB/AA", + "repeats": 1 + }, + { + "count": 524288, + "lower_bound": "A4AAAAAAB/AB", + "upper_bound": "A4AAAAAACAAA", + "repeats": 1 + }, + { + "count": 528384, + "lower_bound": "A4AAAAAACAAB", + "upper_bound": "A4AAAAAACBAA", + "repeats": 1 + }, + { + "count": 532480, + "lower_bound": "A4AAAAAACBAB", + "upper_bound": "A4AAAAAACCAA", + "repeats": 1 + }, + { + "count": 536576, + "lower_bound": "A4AAAAAACCAB", + "upper_bound": "A4AAAAAACDAA", + "repeats": 1 + }, + { + "count": 540672, + "lower_bound": "A4AAAAAACDAB", + "upper_bound": "A4AAAAAACEAA", + "repeats": 1 + }, + { + "count": 544768, + "lower_bound": "A4AAAAAACEAB", + "upper_bound": "A4AAAAAACFAA", + "repeats": 1 + }, + { + "count": 548864, + "lower_bound": "A4AAAAAACFAB", + "upper_bound": "A4AAAAAACGAA", + "repeats": 1 + }, + { + "count": 552960, + "lower_bound": "A4AAAAAACGAB", + "upper_bound": "A4AAAAAACHAA", + "repeats": 1 + }, + { + "count": 557056, + "lower_bound": "A4AAAAAACHAB", + "upper_bound": "A4AAAAAACIAA", + "repeats": 1 + }, + { + "count": 561152, + "lower_bound": "A4AAAAAACIAB", + "upper_bound": "A4AAAAAACJAA", + "repeats": 1 + }, + { + "count": 565248, + "lower_bound": "A4AAAAAACJAB", + "upper_bound": "A4AAAAAACKAA", + "repeats": 1 + }, + { + "count": 569344, + "lower_bound": "A4AAAAAACKAB", + "upper_bound": "A4AAAAAACLAA", + "repeats": 1 + }, + { + "count": 573440, + "lower_bound": "A4AAAAAACLAB", + "upper_bound": "A4AAAAAACMAA", + "repeats": 1 + }, + { + "count": 577536, + "lower_bound": "A4AAAAAACMAB", + "upper_bound": "A4AAAAAACNAA", + "repeats": 1 + }, + { + "count": 581632, + "lower_bound": "A4AAAAAACNAB", + "upper_bound": "A4AAAAAACOAA", + "repeats": 1 + }, + { + "count": 585728, + "lower_bound": "A4AAAAAACOAB", + "upper_bound": "A4AAAAAACPAA", + "repeats": 1 + }, + { + "count": 589824, + "lower_bound": "A4AAAAAACPAB", + "upper_bound": "A4AAAAAACQAA", + "repeats": 1 + }, + { + "count": 593920, + "lower_bound": "A4AAAAAACQAB", + "upper_bound": "A4AAAAAACRAA", + "repeats": 1 + }, + { + "count": 598016, + "lower_bound": "A4AAAAAACRAB", + "upper_bound": "A4AAAAAACSAA", + "repeats": 1 + }, + { + "count": 602112, + "lower_bound": "A4AAAAAACSAB", + "upper_bound": "A4AAAAAACTAA", + "repeats": 1 + }, + { + "count": 606208, + "lower_bound": "A4AAAAAACTAB", + "upper_bound": "A4AAAAAACUAA", + "repeats": 1 + }, + { + "count": 610304, + "lower_bound": "A4AAAAAACUAB", + "upper_bound": "A4AAAAAACVAA", + "repeats": 1 + }, + { + "count": 614400, + "lower_bound": "A4AAAAAACVAB", + "upper_bound": "A4AAAAAACWAA", + "repeats": 1 + }, + { + "count": 618496, + "lower_bound": "A4AAAAAACWAB", + "upper_bound": "A4AAAAAACXAA", + "repeats": 1 + }, + { + "count": 622592, + "lower_bound": "A4AAAAAACXAB", + "upper_bound": "A4AAAAAACYAA", + "repeats": 1 + }, + { + "count": 626688, + "lower_bound": "A4AAAAAACYAB", + "upper_bound": "A4AAAAAACZAA", + "repeats": 1 + }, + { + "count": 630784, + "lower_bound": "A4AAAAAACZAB", + "upper_bound": "A4AAAAAACaAA", + "repeats": 1 + }, + { + "count": 634880, + "lower_bound": "A4AAAAAACaAB", + "upper_bound": "A4AAAAAACbAA", + "repeats": 1 + }, + { + "count": 638976, + "lower_bound": "A4AAAAAACbAB", + "upper_bound": "A4AAAAAACcAA", + "repeats": 1 + }, + { + "count": 643072, + "lower_bound": "A4AAAAAACcAB", + "upper_bound": "A4AAAAAACdAA", + "repeats": 1 + }, + { + "count": 647168, + "lower_bound": "A4AAAAAACdAB", + "upper_bound": "A4AAAAAACeAA", + "repeats": 1 + }, + { + "count": 651264, + "lower_bound": "A4AAAAAACeAB", + "upper_bound": "A4AAAAAACfAA", + "repeats": 1 + }, + { + "count": 655360, + "lower_bound": "A4AAAAAACfAB", + "upper_bound": "A4AAAAAACgAA", + "repeats": 1 + }, + { + "count": 659456, + "lower_bound": "A4AAAAAACgAB", + "upper_bound": "A4AAAAAAChAA", + "repeats": 1 + }, + { + "count": 663552, + "lower_bound": "A4AAAAAAChAB", + "upper_bound": "A4AAAAAACiAA", + "repeats": 1 + }, + { + "count": 667648, + "lower_bound": "A4AAAAAACiAB", + "upper_bound": "A4AAAAAACjAA", + "repeats": 1 + }, + { + "count": 671744, + "lower_bound": "A4AAAAAACjAB", + "upper_bound": "A4AAAAAACkAA", + "repeats": 1 + }, + { + "count": 675840, + "lower_bound": "A4AAAAAACkAB", + "upper_bound": "A4AAAAAAClAA", + "repeats": 1 + }, + { + "count": 679936, + "lower_bound": "A4AAAAAAClAB", + "upper_bound": "A4AAAAAACmAA", + "repeats": 1 + }, + { + "count": 684032, + "lower_bound": "A4AAAAAACmAB", + "upper_bound": "A4AAAAAACnAA", + "repeats": 1 + }, + { + "count": 688128, + "lower_bound": "A4AAAAAACnAB", + "upper_bound": "A4AAAAAACoAA", + "repeats": 1 + }, + { + "count": 692224, + "lower_bound": "A4AAAAAACoAB", + "upper_bound": "A4AAAAAACpAA", + "repeats": 1 + }, + { + "count": 696320, + "lower_bound": "A4AAAAAACpAB", + "upper_bound": "A4AAAAAACqAA", + "repeats": 1 + }, + { + "count": 700416, + "lower_bound": "A4AAAAAACqAB", + "upper_bound": "A4AAAAAACrAA", + "repeats": 1 + }, + { + "count": 704512, + "lower_bound": "A4AAAAAACrAB", + "upper_bound": "A4AAAAAACsAA", + "repeats": 1 + }, + { + "count": 708608, + "lower_bound": "A4AAAAAACsAB", + "upper_bound": "A4AAAAAACtAA", + "repeats": 1 + }, + { + "count": 712704, + "lower_bound": "A4AAAAAACtAB", + "upper_bound": "A4AAAAAACuAA", + "repeats": 1 + }, + { + "count": 716800, + "lower_bound": "A4AAAAAACuAB", + "upper_bound": "A4AAAAAACvAA", + "repeats": 1 + }, + { + "count": 720896, + "lower_bound": "A4AAAAAACvAB", + "upper_bound": "A4AAAAAACwAA", + "repeats": 1 + }, + { + "count": 724992, + "lower_bound": "A4AAAAAACwAB", + "upper_bound": "A4AAAAAACxAA", + "repeats": 1 + }, + { + "count": 729088, + "lower_bound": "A4AAAAAACxAB", + "upper_bound": "A4AAAAAACyAA", + "repeats": 1 + }, + { + "count": 733184, + "lower_bound": "A4AAAAAACyAB", + "upper_bound": "A4AAAAAACzAA", + "repeats": 1 + }, + { + "count": 737280, + "lower_bound": "A4AAAAAACzAB", + "upper_bound": "A4AAAAAAC0AA", + "repeats": 1 + }, + { + "count": 741376, + "lower_bound": "A4AAAAAAC0AB", + "upper_bound": "A4AAAAAAC1AA", + "repeats": 1 + }, + { + "count": 745472, + "lower_bound": "A4AAAAAAC1AB", + "upper_bound": "A4AAAAAAC2AA", + "repeats": 1 + }, + { + "count": 749568, + "lower_bound": "A4AAAAAAC2AB", + "upper_bound": "A4AAAAAAC3AA", + "repeats": 1 + }, + { + "count": 753664, + "lower_bound": "A4AAAAAAC3AB", + "upper_bound": "A4AAAAAAC4AA", + "repeats": 1 + }, + { + "count": 757760, + "lower_bound": "A4AAAAAAC4AB", + "upper_bound": "A4AAAAAAC5AA", + "repeats": 1 + }, + { + "count": 761856, + "lower_bound": "A4AAAAAAC5AB", + "upper_bound": "A4AAAAAAC6AA", + "repeats": 1 + }, + { + "count": 765952, + "lower_bound": "A4AAAAAAC6AB", + "upper_bound": "A4AAAAAAC7AA", + "repeats": 1 + }, + { + "count": 770048, + "lower_bound": "A4AAAAAAC7AB", + "upper_bound": "A4AAAAAAC8AA", + "repeats": 1 + }, + { + "count": 774144, + "lower_bound": "A4AAAAAAC8AB", + "upper_bound": "A4AAAAAAC9AA", + "repeats": 1 + }, + { + "count": 778240, + "lower_bound": "A4AAAAAAC9AB", + "upper_bound": "A4AAAAAAC+AA", + "repeats": 1 + }, + { + "count": 782336, + "lower_bound": "A4AAAAAAC+AB", + "upper_bound": "A4AAAAAAC/AA", + "repeats": 1 + }, + { + "count": 786432, + "lower_bound": "A4AAAAAAC/AB", + "upper_bound": "A4AAAAAADAAA", + "repeats": 1 + }, + { + "count": 790528, + "lower_bound": "A4AAAAAADAAB", + "upper_bound": "A4AAAAAADBAA", + "repeats": 1 + }, + { + "count": 794624, + "lower_bound": "A4AAAAAADBAB", + "upper_bound": "A4AAAAAADCAA", + "repeats": 1 + }, + { + "count": 798720, + "lower_bound": "A4AAAAAADCAB", + "upper_bound": "A4AAAAAADDAA", + "repeats": 1 + }, + { + "count": 802816, + "lower_bound": "A4AAAAAADDAB", + "upper_bound": "A4AAAAAADEAA", + "repeats": 1 + }, + { + "count": 806912, + "lower_bound": "A4AAAAAADEAB", + "upper_bound": "A4AAAAAADFAA", + "repeats": 1 + }, + { + "count": 811008, + "lower_bound": "A4AAAAAADFAB", + "upper_bound": "A4AAAAAADGAA", + "repeats": 1 + }, + { + "count": 815104, + "lower_bound": "A4AAAAAADGAB", + "upper_bound": "A4AAAAAADHAA", + "repeats": 1 + }, + { + "count": 819200, + "lower_bound": "A4AAAAAADHAB", + "upper_bound": "A4AAAAAADIAA", + "repeats": 1 + }, + { + "count": 823296, + "lower_bound": "A4AAAAAADIAB", + "upper_bound": "A4AAAAAADJAA", + "repeats": 1 + }, + { + "count": 827392, + "lower_bound": "A4AAAAAADJAB", + "upper_bound": "A4AAAAAADKAA", + "repeats": 1 + }, + { + "count": 831488, + "lower_bound": "A4AAAAAADKAB", + "upper_bound": "A4AAAAAADLAA", + "repeats": 1 + }, + { + "count": 835584, + "lower_bound": "A4AAAAAADLAB", + "upper_bound": "A4AAAAAADMAA", + "repeats": 1 + }, + { + "count": 839680, + "lower_bound": "A4AAAAAADMAB", + "upper_bound": "A4AAAAAADNAA", + "repeats": 1 + }, + { + "count": 843776, + "lower_bound": "A4AAAAAADNAB", + "upper_bound": "A4AAAAAADOAA", + "repeats": 1 + }, + { + "count": 847872, + "lower_bound": "A4AAAAAADOAB", + "upper_bound": "A4AAAAAADPAA", + "repeats": 1 + }, + { + "count": 851968, + "lower_bound": "A4AAAAAADPAB", + "upper_bound": "A4AAAAAADQAA", + "repeats": 1 + }, + { + "count": 856064, + "lower_bound": "A4AAAAAADQAB", + "upper_bound": "A4AAAAAADRAA", + "repeats": 1 + }, + { + "count": 860160, + "lower_bound": "A4AAAAAADRAB", + "upper_bound": "A4AAAAAADSAA", + "repeats": 1 + }, + { + "count": 864256, + "lower_bound": "A4AAAAAADSAB", + "upper_bound": "A4AAAAAADTAA", + "repeats": 1 + }, + { + "count": 868352, + "lower_bound": "A4AAAAAADTAB", + "upper_bound": "A4AAAAAADUAA", + "repeats": 1 + }, + { + "count": 872448, + "lower_bound": "A4AAAAAADUAB", + "upper_bound": "A4AAAAAADVAA", + "repeats": 1 + }, + { + "count": 876544, + "lower_bound": "A4AAAAAADVAB", + "upper_bound": "A4AAAAAADWAA", + "repeats": 1 + }, + { + "count": 880640, + "lower_bound": "A4AAAAAADWAB", + "upper_bound": "A4AAAAAADXAA", + "repeats": 1 + }, + { + "count": 883011, + "lower_bound": "A4AAAAAADXAB", + "upper_bound": "A4AAAAAADXlD", + "repeats": 1 + }, + { + "count": 887107, + "lower_bound": "A4AAAAAADXlE", + "upper_bound": "A4AAAAAADYlD", + "repeats": 1 + }, + { + "count": 891203, + "lower_bound": "A4AAAAAADYlE", + "upper_bound": "A4AAAAAADZlD", + "repeats": 1 + }, + { + "count": 895299, + "lower_bound": "A4AAAAAADZlE", + "upper_bound": "A4AAAAAADalD", + "repeats": 1 + }, + { + "count": 899395, + "lower_bound": "A4AAAAAADalE", + "upper_bound": "A4AAAAAADblD", + "repeats": 1 + }, + { + "count": 903491, + "lower_bound": "A4AAAAAADblE", + "upper_bound": "A4AAAAAADclD", + "repeats": 1 + }, + { + "count": 907587, + "lower_bound": "A4AAAAAADclE", + "upper_bound": "A4AAAAAADdlD", + "repeats": 1 + }, + { + "count": 911683, + "lower_bound": "A4AAAAAADdlE", + "upper_bound": "A4AAAAAADelD", + "repeats": 1 + }, + { + "count": 915779, + "lower_bound": "A4AAAAAADelE", + "upper_bound": "A4AAAAAADflD", + "repeats": 1 + }, + { + "count": 919875, + "lower_bound": "A4AAAAAADflE", + "upper_bound": "A4AAAAAADglD", + "repeats": 1 + }, + { + "count": 923971, + "lower_bound": "A4AAAAAADglE", + "upper_bound": "A4AAAAAADhlD", + "repeats": 1 + }, + { + "count": 928067, + "lower_bound": "A4AAAAAADhlE", + "upper_bound": "A4AAAAAADilD", + "repeats": 1 + }, + { + "count": 932163, + "lower_bound": "A4AAAAAADilE", + "upper_bound": "A4AAAAAADjlD", + "repeats": 1 + }, + { + "count": 936259, + "lower_bound": "A4AAAAAADjlE", + "upper_bound": "A4AAAAAADklD", + "repeats": 1 + }, + { + "count": 940355, + "lower_bound": "A4AAAAAADklE", + "upper_bound": "A4AAAAAADllD", + "repeats": 1 + }, + { + "count": 944451, + "lower_bound": "A4AAAAAADllE", + "upper_bound": "A4AAAAAADmlD", + "repeats": 1 + }, + { + "count": 948547, + "lower_bound": "A4AAAAAADmlE", + "upper_bound": "A4AAAAAADnlD", + "repeats": 1 + }, + { + "count": 952643, + "lower_bound": "A4AAAAAADnlE", + "upper_bound": "A4AAAAAADolD", + "repeats": 1 + }, + { + "count": 956739, + "lower_bound": "A4AAAAAADolE", + "upper_bound": "A4AAAAAADplD", + "repeats": 1 + }, + { + "count": 960835, + "lower_bound": "A4AAAAAADplE", + "upper_bound": "A4AAAAAADqlD", + "repeats": 1 + }, + { + "count": 964931, + "lower_bound": "A4AAAAAADqlE", + "upper_bound": "A4AAAAAADrlD", + "repeats": 1 + }, + { + "count": 969027, + "lower_bound": "A4AAAAAADrlE", + "upper_bound": "A4AAAAAADslD", + "repeats": 1 + }, + { + "count": 973123, + "lower_bound": "A4AAAAAADslE", + "upper_bound": "A4AAAAAADtlD", + "repeats": 1 + }, + { + "count": 977219, + "lower_bound": "A4AAAAAADtlE", + "upper_bound": "A4AAAAAADulD", + "repeats": 1 + }, + { + "count": 981315, + "lower_bound": "A4AAAAAADulE", + "upper_bound": "A4AAAAAADvlD", + "repeats": 1 + }, + { + "count": 985411, + "lower_bound": "A4AAAAAADvlE", + "upper_bound": "A4AAAAAADwlD", + "repeats": 1 + }, + { + "count": 989507, + "lower_bound": "A4AAAAAADwlE", + "upper_bound": "A4AAAAAADxlD", + "repeats": 1 + }, + { + "count": 993603, + "lower_bound": "A4AAAAAADxlE", + "upper_bound": "A4AAAAAADylD", + "repeats": 1 + }, + { + "count": 997699, + "lower_bound": "A4AAAAAADylE", + "upper_bound": "A4AAAAAADzlD", + "repeats": 1 + }, + { + "count": 1000000, + "lower_bound": "A4AAAAAADzlE", + "upper_bound": "A4AAAAAAD0JA", + "repeats": 1 + } + ] + }, + "cm_sketch": { + "rows": [ + { + "counters": [ + 513, + 507, + 485, + 476, + 515, + 490, + 516, + 480, + 481, + 477, + 533, + 481, + 467, + 489, + 504, + 484, + 514, + 477, + 475, + 511, + 529, + 469, + 474, + 486, + 478, + 497, + 503, + 547, + 466, + 470, + 504, + 468, + 511, + 476, + 501, + 478, + 520, + 454, + 521, + 467, + 508, + 449, + 467, + 460, + 466, + 464, + 511, + 455, + 492, + 535, + 478, + 478, + 473, + 519, + 492, + 484, + 480, + 495, + 496, + 502, + 498, + 439, + 457, + 535, + 506, + 536, + 501, + 492, + 484, + 450, + 498, + 467, + 491, + 484, + 465, + 503, + 489, + 477, + 514, + 491, + 517, + 459, + 467, + 463, + 499, + 466, + 488, + 525, + 486, + 520, + 528, + 490, + 478, + 482, + 476, + 456, + 477, + 522, + 519, + 483, + 441, + 516, + 455, + 481, + 474, + 469, + 480, + 487, + 508, + 514, + 515, + 452, + 453, + 484, + 495, + 470, + 496, + 490, + 515, + 487, + 503, + 509, + 480, + 495, + 480, + 504, + 512, + 526, + 462, + 472, + 488, + 510, + 488, + 498, + 486, + 462, + 486, + 493, + 509, + 527, + 510, + 477, + 507, + 474, + 486, + 473, + 493, + 480, + 522, + 513, + 453, + 472, + 467, + 474, + 463, + 504, + 482, + 514, + 498, + 514, + 523, + 500, + 453, + 458, + 480, + 483, + 513, + 460, + 524, + 502, + 478, + 477, + 447, + 511, + 495, + 509, + 503, + 487, + 491, + 494, + 470, + 481, + 460, + 488, + 490, + 488, + 523, + 535, + 489, + 489, + 493, + 516, + 515, + 469, + 488, + 541, + 500, + 489, + 477, + 483, + 465, + 478, + 486, + 469, + 492, + 453, + 478, + 464, + 515, + 508, + 466, + 498, + 479, + 478, + 458, + 452, + 456, + 480, + 527, + 461, + 494, + 467, + 471, + 498, + 496, + 488, + 454, + 473, + 505, + 499, + 489, + 487, + 503, + 477, + 469, + 493, + 459, + 503, + 476, + 483, + 497, + 488, + 469, + 504, + 523, + 479, + 528, + 508, + 510, + 474, + 504, + 458, + 483, + 506, + 464, + 512, + 482, + 472, + 487, + 492, + 477, + 487, + 490, + 477, + 483, + 471, + 455, + 465, + 450, + 494, + 501, + 497, + 460, + 493, + 497, + 464, + 504, + 513, + 499, + 491, + 477, + 504, + 515, + 525, + 474, + 449, + 467, + 492, + 470, + 473, + 484, + 458, + 494, + 460, + 464, + 492, + 462, + 495, + 490, + 501, + 489, + 502, + 488, + 483, + 473, + 499, + 477, + 467, + 510, + 509, + 484, + 481, + 500, + 480, + 493, + 460, + 497, + 463, + 475, + 530, + 513, + 476, + 443, + 516, + 439, + 522, + 510, + 509, + 525, + 495, + 509, + 461, + 489, + 479, + 468, + 459, + 474, + 461, + 475, + 469, + 470, + 466, + 495, + 481, + 462, + 494, + 501, + 503, + 495, + 489, + 470, + 486, + 478, + 515, + 474, + 463, + 460, + 456, + 507, + 502, + 471, + 471, + 487, + 470, + 475, + 504, + 514, + 475, + 492, + 489, + 487, + 520, + 476, + 482, + 477, + 468, + 475, + 495, + 477, + 478, + 468, + 473, + 471, + 505, + 533, + 500, + 479, + 487, + 457, + 497, + 471, + 488, + 487, + 451, + 500, + 515, + 507, + 485, + 508, + 458, + 485, + 488, + 470, + 496, + 463, + 479, + 491, + 490, + 542, + 496, + 492, + 492, + 501, + 512, + 467, + 490, + 484, + 507, + 489, + 502, + 467, + 468, + 499, + 494, + 528, + 464, + 482, + 521, + 514, + 451, + 528, + 485, + 516, + 513, + 497, + 483, + 468, + 492, + 510, + 473, + 457, + 503, + 500, + 494, + 497, + 510, + 456, + 488, + 472, + 500, + 490, + 487, + 473, + 468, + 476, + 471, + 523, + 479, + 475, + 499, + 517, + 520, + 477, + 488, + 512, + 461, + 490, + 464, + 506, + 496, + 522, + 493, + 450, + 498, + 489, + 480, + 483, + 558, + 453, + 523, + 481, + 480, + 446, + 489, + 461, + 505, + 514, + 505, + 519, + 504, + 484, + 483, + 472, + 458, + 469, + 489, + 493, + 473, + 498, + 492, + 528, + 449, + 465, + 475, + 487, + 512, + 508, + 504, + 472, + 480, + 462, + 496, + 457, + 506, + 480, + 462, + 542, + 563, + 480, + 480, + 457, + 502, + 465, + 475, + 474, + 450, + 476, + 508, + 452, + 464, + 496, + 465, + 441, + 501, + 455, + 494, + 507, + 522, + 484, + 473, + 511, + 473, + 483, + 489, + 474, + 479, + 447, + 492, + 468, + 496, + 480, + 530, + 502, + 484, + 481, + 538, + 488, + 465, + 527, + 455, + 506, + 484, + 497, + 505, + 476, + 466, + 456, + 476, + 480, + 476, + 503, + 502, + 506, + 485, + 466, + 462, + 508, + 507, + 475, + 470, + 511, + 487, + 468, + 469, + 474, + 499, + 465, + 473, + 480, + 462, + 478, + 466, + 427, + 477, + 491, + 484, + 492, + 481, + 519, + 484, + 448, + 481, + 488, + 506, + 506, + 477, + 493, + 512, + 512, + 508, + 494, + 505, + 479, + 474, + 481, + 451, + 496, + 494, + 482, + 466, + 487, + 488, + 497, + 439, + 475, + 499, + 460, + 465, + 508, + 443, + 496, + 492, + 504, + 450, + 514, + 490, + 491, + 505, + 501, + 476, + 480, + 484, + 513, + 486, + 467, + 482, + 524, + 486, + 500, + 431, + 531, + 515, + 475, + 491, + 483, + 487, + 488, + 440, + 508, + 507, + 524, + 472, + 466, + 524, + 471, + 484, + 438, + 517, + 470, + 486, + 490, + 515, + 482, + 501, + 467, + 491, + 458, + 483, + 502, + 463, + 503, + 496, + 496, + 480, + 478, + 521, + 493, + 496, + 490, + 494, + 485, + 510, + 461, + 480, + 487, + 512, + 475, + 452, + 483, + 481, + 478, + 531, + 497, + 515, + 505, + 462, + 510, + 513, + 510, + 528, + 509, + 446, + 500, + 510, + 473, + 494, + 450, + 464, + 465, + 470, + 477, + 478, + 493, + 495, + 495, + 470, + 501, + 519, + 500, + 432, + 446, + 511, + 530, + 455, + 446, + 495, + 452, + 456, + 480, + 467, + 454, + 485, + 504, + 434, + 519, + 490, + 460, + 491, + 501, + 493, + 482, + 470, + 501, + 477, + 516, + 480, + 490, + 455, + 514, + 501, + 488, + 505, + 504, + 507, + 532, + 490, + 502, + 473, + 516, + 450, + 510, + 475, + 495, + 482, + 473, + 449, + 504, + 526, + 535, + 471, + 447, + 493, + 507, + 527, + 491, + 492, + 500, + 470, + 454, + 441, + 483, + 488, + 463, + 474, + 499, + 486, + 485, + 486, + 480, + 461, + 500, + 466, + 494, + 494, + 480, + 518, + 508, + 469, + 498, + 455, + 486, + 480, + 487, + 502, + 502, + 458, + 511, + 481, + 483, + 489, + 451, + 469, + 495, + 491, + 487, + 482, + 509, + 455, + 501, + 480, + 473, + 519, + 466, + 475, + 493, + 510, + 495, + 486, + 450, + 562, + 502, + 462, + 485, + 485, + 512, + 513, + 473, + 515, + 502, + 464, + 497, + 438, + 514, + 507, + 438, + 473, + 533, + 506, + 460, + 457, + 437, + 516, + 470, + 470, + 499, + 456, + 502, + 489, + 498, + 493, + 528, + 507, + 529, + 468, + 512, + 500, + 506, + 499, + 459, + 497, + 512, + 487, + 505, + 489, + 492, + 501, + 494, + 487, + 443, + 519, + 478, + 467, + 506, + 493, + 511, + 504, + 472, + 479, + 466, + 517, + 477, + 532, + 473, + 501, + 494, + 467, + 498, + 542, + 496, + 461, + 486, + 480, + 513, + 482, + 474, + 498, + 484, + 509, + 499, + 460, + 499, + 474, + 485, + 458, + 501, + 477, + 487, + 478, + 461, + 465, + 492, + 530, + 485, + 512, + 487, + 503, + 492, + 473, + 503, + 468, + 503, + 454, + 488, + 504, + 528, + 490, + 471, + 530, + 507, + 488, + 507, + 503, + 488, + 482, + 494, + 449, + 505, + 490, + 458, + 496, + 480, + 511, + 467, + 512, + 465, + 486, + 513, + 480, + 468, + 487, + 487, + 466, + 489, + 460, + 465, + 504, + 481, + 510, + 464, + 526, + 467, + 491, + 553, + 510, + 519, + 456, + 491, + 525, + 516, + 533, + 469, + 544, + 508, + 515, + 482, + 480, + 465, + 505, + 479, + 500, + 456, + 503, + 487, + 496, + 512, + 497, + 464, + 490, + 513, + 506, + 509, + 487, + 483, + 494, + 509, + 510, + 510, + 461, + 493, + 419, + 501, + 474, + 487, + 482, + 499, + 480, + 447, + 470, + 507, + 473, + 498, + 517, + 515, + 497, + 459, + 482, + 483, + 497, + 487, + 505, + 509, + 495, + 468, + 512, + 477, + 524, + 526, + 493, + 516, + 484, + 500, + 477, + 527, + 475, + 466, + 512, + 497, + 477, + 539, + 486, + 483, + 512, + 491, + 486, + 502, + 522, + 483, + 529, + 468, + 470, + 517, + 509, + 512, + 522, + 500, + 497, + 506, + 489, + 451, + 487, + 456, + 486, + 455, + 468, + 467, + 487, + 509, + 474, + 499, + 459, + 471, + 469, + 456, + 482, + 463, + 479, + 495, + 530, + 487, + 501, + 488, + 504, + 512, + 545, + 496, + 491, + 482, + 522, + 491, + 517, + 466, + 509, + 516, + 472, + 503, + 472, + 494, + 478, + 481, + 508, + 466, + 454, + 452, + 501, + 483, + 438, + 528, + 469, + 474, + 516, + 464, + 453, + 471, + 468, + 477, + 458, + 485, + 479, + 473, + 497, + 498, + 495, + 498, + 502, + 511, + 523, + 507, + 455, + 492, + 516, + 473, + 496, + 489, + 524, + 502, + 508, + 510, + 530, + 527, + 482, + 475, + 526, + 484, + 474, + 500, + 521, + 484, + 478, + 493, + 505, + 491, + 513, + 488, + 512, + 455, + 541, + 509, + 486, + 493, + 438, + 449, + 491, + 476, + 476, + 474, + 503, + 515, + 498, + 511, + 477, + 444, + 478, + 534, + 471, + 488, + 501, + 475, + 454, + 476, + 480, + 472, + 470, + 514, + 510, + 483, + 471, + 507, + 497, + 469, + 497, + 473, + 526, + 468, + 477, + 480, + 496, + 500, + 525, + 450, + 477, + 450, + 516, + 520, + 519, + 485, + 510, + 456, + 516, + 486, + 473, + 484, + 445, + 513, + 489, + 499, + 479, + 479, + 449, + 500, + 463, + 469, + 504, + 483, + 503, + 489, + 505, + 463, + 476, + 485, + 440, + 501, + 454, + 514, + 453, + 510, + 481, + 500, + 487, + 525, + 482, + 460, + 512, + 512, + 499, + 505, + 508, + 471, + 504, + 491, + 508, + 497, + 500, + 486, + 486, + 496, + 446, + 471, + 500, + 444, + 486, + 483, + 505, + 497, + 483, + 462, + 495, + 480, + 467, + 498, + 476, + 492, + 466, + 509, + 462, + 516, + 542, + 488, + 449, + 480, + 488, + 508, + 499, + 499, + 504, + 498, + 493, + 456, + 457, + 484, + 496, + 478, + 552, + 484, + 513, + 482, + 518, + 503, + 490, + 492, + 450, + 478, + 504, + 504, + 488, + 476, + 520, + 497, + 485, + 514, + 504, + 496, + 499, + 484, + 493, + 475, + 505, + 476, + 490, + 490, + 474, + 497, + 472, + 493, + 460, + 496, + 483, + 498, + 496, + 508, + 449, + 497, + 483, + 480, + 470, + 487, + 491, + 485, + 485, + 530, + 497, + 492, + 484, + 471, + 489, + 467, + 495, + 460, + 483, + 501, + 440, + 496, + 444, + 501, + 443, + 519, + 499, + 499, + 513, + 512, + 496, + 478, + 523, + 485, + 465, + 524, + 484, + 494, + 489, + 508, + 481, + 495, + 474, + 488, + 449, + 475, + 524, + 502, + 447, + 501, + 510, + 532, + 496, + 476, + 502, + 455, + 478, + 485, + 551, + 482, + 544, + 488, + 479, + 513, + 530, + 483, + 520, + 494, + 487, + 478, + 472, + 499, + 477, + 487, + 500, + 465, + 515, + 499, + 465, + 511, + 493, + 493, + 473, + 515, + 468, + 459, + 450, + 472, + 456, + 511, + 493, + 490, + 493, + 471, + 474, + 485, + 474, + 503, + 499, + 502, + 491, + 500, + 501, + 503, + 490, + 484, + 480, + 498, + 448, + 483, + 499, + 516, + 488, + 487, + 459, + 481, + 511, + 480, + 479, + 464, + 477, + 492, + 488, + 422, + 508, + 477, + 478, + 439, + 501, + 495, + 523, + 445, + 497, + 510, + 455, + 455, + 504, + 461, + 479, + 436, + 471, + 490, + 499, + 520, + 477, + 458, + 488, + 488, + 561, + 487, + 484, + 517, + 465, + 483, + 498, + 472, + 496, + 479, + 442, + 489, + 498, + 495, + 533, + 434, + 506, + 539, + 487, + 493, + 495, + 484, + 465, + 506, + 473, + 475, + 475, + 510, + 455, + 474, + 486, + 519, + 490, + 498, + 465, + 498, + 501, + 511, + 525, + 543, + 478, + 446, + 490, + 491, + 477, + 495, + 494, + 498, + 499, + 457, + 500, + 518, + 507, + 488, + 484, + 489, + 486, + 516, + 468, + 502, + 503, + 523, + 506, + 518, + 459, + 446, + 458, + 519, + 507, + 486, + 504, + 498, + 507, + 500, + 475, + 514, + 452, + 500, + 484, + 508, + 471, + 468, + 461, + 485, + 503, + 480, + 465, + 476, + 464, + 502, + 510, + 541, + 484, + 477, + 477, + 491, + 480, + 480, + 489, + 541, + 536, + 463, + 484, + 489, + 459, + 486, + 467, + 499, + 486, + 471, + 546, + 476, + 524, + 491, + 461, + 486, + 486, + 461, + 476, + 511, + 512, + 495, + 499, + 509, + 478, + 480, + 504, + 488, + 515, + 478, + 495, + 463, + 473, + 480, + 489, + 505, + 498, + 526, + 467, + 483, + 490, + 478, + 491, + 509, + 505, + 484, + 531, + 474, + 508, + 433, + 510, + 495, + 439, + 497, + 500, + 466, + 492, + 489, + 439, + 508, + 482, + 508, + 460, + 478, + 517, + 458, + 477, + 463, + 505, + 494, + 493, + 507, + 494, + 504, + 491, + 449, + 492, + 433, + 515, + 467, + 514, + 462, + 459, + 474, + 472, + 489, + 504, + 472, + 466, + 489, + 479, + 507, + 499, + 448, + 477, + 453, + 471, + 450, + 500, + 490, + 450, + 491, + 463, + 501, + 494, + 500, + 469, + 486, + 516, + 497, + 494, + 462, + 498, + 500, + 470, + 499, + 464, + 506, + 504, + 513, + 479, + 473, + 479, + 484, + 535, + 483, + 518, + 512, + 501, + 461, + 476, + 504, + 470, + 498, + 466, + 494, + 470, + 500, + 503, + 479, + 444, + 510, + 497, + 501, + 509, + 468, + 495, + 456, + 470, + 492, + 521, + 475, + 491, + 488, + 465, + 443, + 505, + 520, + 485, + 499, + 524, + 522, + 496, + 490, + 481, + 451, + 492, + 489, + 518, + 484, + 467, + 469, + 524, + 512, + 481, + 484, + 481, + 518, + 438, + 517, + 515, + 491, + 485, + 467, + 495, + 495, + 488, + 487, + 484, + 510, + 483, + 525, + 507, + 486, + 449, + 429, + 468, + 494, + 463, + 491, + 523, + 491, + 460, + 498, + 445, + 454, + 479, + 462, + 558, + 482, + 502, + 517, + 444, + 481, + 498, + 527, + 508, + 484, + 439, + 474, + 485, + 447, + 484, + 536, + 453, + 441, + 481, + 510, + 438, + 463, + 476, + 476, + 486, + 508, + 477, + 492, + 490, + 473, + 486, + 472, + 510, + 509, + 473, + 478, + 518, + 528, + 494, + 499, + 499, + 477, + 480, + 480, + 472, + 492, + 514, + 470, + 495, + 476, + 510, + 467, + 496, + 522, + 508, + 493, + 476, + 515, + 524, + 485, + 487, + 492, + 494, + 476, + 501, + 510, + 508, + 457, + 473, + 544, + 499, + 507, + 545, + 477, + 508, + 503, + 493, + 481, + 488, + 537, + 492, + 503, + 471, + 479, + 512, + 496, + 511, + 486, + 507, + 459, + 472, + 469, + 454, + 520, + 512, + 473, + 480, + 472, + 501, + 466, + 525, + 518, + 444, + 488, + 467, + 478, + 471, + 451, + 516, + 439, + 482, + 482, + 493, + 500, + 472, + 499, + 487, + 493, + 509, + 463, + 504, + 506, + 492, + 468, + 487, + 463, + 468, + 457, + 495, + 475, + 512, + 499, + 528, + 487, + 487, + 485, + 502, + 468, + 446, + 490, + 499, + 482, + 495, + 454, + 507, + 480, + 451, + 464, + 495, + 471, + 531, + 456, + 464, + 470, + 495, + 525, + 527, + 465, + 530, + 501, + 489, + 504, + 475, + 521, + 522, + 488, + 484, + 470, + 532, + 475, + 548, + 469, + 484, + 495, + 482, + 482, + 466, + 472, + 508, + 456, + 512, + 452, + 470, + 469, + 460, + 524, + 499, + 481, + 468, + 509, + 467, + 514, + 487, + 468, + 530, + 483, + 482, + 474, + 498, + 500, + 462, + 525, + 451, + 504, + 494, + 531, + 489, + 503, + 444, + 491, + 478, + 503, + 443, + 482, + 481, + 515, + 543, + 483, + 484, + 508, + 453, + 454, + 517, + 508, + 462, + 477, + 527, + 451, + 529, + 512, + 484, + 475, + 509, + 456, + 519, + 493, + 500, + 497, + 459, + 485, + 514, + 514, + 522 + ] + }, + { + "counters": [ + 472, + 498, + 470, + 465, + 475, + 525, + 470, + 465, + 469, + 495, + 486, + 460, + 487, + 480, + 495, + 479, + 448, + 477, + 477, + 486, + 461, + 456, + 504, + 474, + 482, + 473, + 492, + 509, + 465, + 523, + 451, + 513, + 502, + 462, + 461, + 488, + 523, + 469, + 513, + 501, + 484, + 521, + 463, + 487, + 510, + 446, + 528, + 529, + 467, + 514, + 493, + 513, + 476, + 490, + 461, + 476, + 483, + 488, + 445, + 482, + 516, + 529, + 471, + 454, + 499, + 496, + 483, + 467, + 504, + 481, + 495, + 488, + 489, + 474, + 498, + 510, + 486, + 497, + 510, + 479, + 491, + 472, + 492, + 500, + 505, + 502, + 519, + 461, + 490, + 514, + 498, + 492, + 524, + 492, + 473, + 474, + 520, + 470, + 444, + 509, + 483, + 510, + 508, + 476, + 484, + 484, + 475, + 468, + 475, + 469, + 466, + 471, + 439, + 493, + 479, + 499, + 468, + 487, + 460, + 513, + 480, + 531, + 500, + 527, + 460, + 490, + 475, + 497, + 508, + 484, + 486, + 487, + 488, + 492, + 520, + 477, + 504, + 470, + 509, + 485, + 496, + 466, + 476, + 479, + 501, + 494, + 495, + 485, + 510, + 499, + 514, + 485, + 494, + 474, + 495, + 465, + 480, + 475, + 479, + 548, + 466, + 517, + 507, + 507, + 458, + 460, + 507, + 471, + 493, + 472, + 497, + 509, + 504, + 498, + 488, + 484, + 470, + 453, + 495, + 485, + 467, + 525, + 485, + 522, + 469, + 500, + 441, + 467, + 509, + 469, + 513, + 452, + 489, + 498, + 497, + 435, + 476, + 500, + 510, + 520, + 475, + 461, + 501, + 513, + 488, + 480, + 494, + 481, + 493, + 501, + 470, + 504, + 475, + 500, + 499, + 471, + 469, + 519, + 484, + 478, + 434, + 448, + 522, + 445, + 489, + 515, + 463, + 511, + 483, + 484, + 468, + 480, + 488, + 482, + 476, + 469, + 462, + 470, + 477, + 500, + 475, + 503, + 480, + 539, + 473, + 498, + 522, + 475, + 502, + 526, + 474, + 464, + 493, + 449, + 527, + 464, + 510, + 511, + 466, + 489, + 493, + 512, + 478, + 498, + 469, + 515, + 529, + 506, + 488, + 506, + 535, + 460, + 509, + 489, + 473, + 478, + 503, + 461, + 484, + 517, + 426, + 469, + 473, + 470, + 477, + 454, + 505, + 495, + 476, + 462, + 495, + 486, + 468, + 488, + 498, + 463, + 468, + 503, + 530, + 541, + 488, + 475, + 497, + 502, + 464, + 516, + 492, + 505, + 476, + 454, + 514, + 492, + 463, + 506, + 488, + 487, + 493, + 497, + 498, + 511, + 517, + 490, + 499, + 474, + 458, + 485, + 490, + 504, + 489, + 487, + 468, + 503, + 483, + 504, + 484, + 499, + 489, + 537, + 507, + 509, + 527, + 521, + 471, + 501, + 459, + 539, + 470, + 491, + 474, + 511, + 479, + 474, + 525, + 510, + 478, + 474, + 476, + 481, + 519, + 462, + 492, + 471, + 504, + 472, + 478, + 483, + 448, + 504, + 519, + 479, + 442, + 472, + 492, + 477, + 545, + 472, + 498, + 420, + 500, + 469, + 495, + 508, + 485, + 502, + 492, + 470, + 468, + 472, + 481, + 470, + 493, + 466, + 496, + 482, + 500, + 513, + 522, + 484, + 496, + 476, + 487, + 503, + 491, + 507, + 461, + 503, + 496, + 478, + 515, + 470, + 505, + 542, + 487, + 502, + 503, + 482, + 454, + 509, + 510, + 460, + 514, + 521, + 465, + 491, + 444, + 471, + 470, + 488, + 495, + 492, + 492, + 514, + 521, + 479, + 492, + 538, + 454, + 482, + 486, + 472, + 505, + 523, + 514, + 516, + 488, + 523, + 505, + 480, + 477, + 503, + 487, + 513, + 463, + 509, + 480, + 515, + 497, + 460, + 532, + 474, + 445, + 450, + 501, + 530, + 507, + 495, + 499, + 492, + 495, + 477, + 487, + 478, + 506, + 476, + 510, + 493, + 440, + 490, + 450, + 522, + 484, + 497, + 494, + 487, + 477, + 482, + 460, + 445, + 474, + 481, + 510, + 504, + 478, + 503, + 484, + 497, + 467, + 537, + 469, + 523, + 521, + 472, + 489, + 464, + 476, + 482, + 527, + 491, + 483, + 476, + 500, + 496, + 505, + 462, + 472, + 453, + 493, + 506, + 458, + 446, + 499, + 477, + 473, + 532, + 495, + 466, + 498, + 484, + 491, + 485, + 486, + 478, + 460, + 513, + 482, + 474, + 470, + 482, + 463, + 490, + 479, + 513, + 496, + 499, + 458, + 458, + 490, + 483, + 508, + 475, + 485, + 486, + 491, + 538, + 493, + 486, + 516, + 496, + 498, + 490, + 483, + 509, + 476, + 497, + 463, + 489, + 488, + 496, + 498, + 475, + 510, + 487, + 457, + 451, + 515, + 554, + 488, + 467, + 461, + 463, + 486, + 488, + 482, + 500, + 512, + 509, + 468, + 467, + 474, + 523, + 488, + 480, + 471, + 479, + 502, + 487, + 465, + 472, + 499, + 478, + 478, + 487, + 479, + 479, + 520, + 520, + 491, + 472, + 511, + 499, + 492, + 469, + 491, + 500, + 497, + 465, + 530, + 492, + 450, + 490, + 451, + 505, + 538, + 491, + 493, + 513, + 468, + 452, + 491, + 487, + 514, + 496, + 461, + 471, + 504, + 488, + 504, + 511, + 502, + 439, + 526, + 483, + 491, + 506, + 503, + 505, + 498, + 498, + 477, + 487, + 456, + 447, + 522, + 517, + 492, + 478, + 485, + 522, + 526, + 491, + 447, + 465, + 491, + 457, + 451, + 490, + 455, + 436, + 435, + 468, + 480, + 491, + 503, + 512, + 472, + 524, + 458, + 502, + 443, + 476, + 483, + 495, + 490, + 490, + 463, + 508, + 476, + 466, + 508, + 479, + 516, + 518, + 513, + 455, + 490, + 532, + 490, + 487, + 453, + 485, + 527, + 480, + 508, + 500, + 459, + 471, + 455, + 519, + 488, + 487, + 492, + 468, + 485, + 488, + 460, + 486, + 502, + 520, + 508, + 483, + 473, + 497, + 486, + 488, + 452, + 434, + 493, + 526, + 483, + 467, + 476, + 488, + 469, + 501, + 500, + 464, + 480, + 526, + 475, + 526, + 422, + 519, + 468, + 503, + 528, + 499, + 500, + 470, + 523, + 504, + 493, + 474, + 482, + 485, + 480, + 527, + 498, + 472, + 516, + 439, + 481, + 474, + 505, + 491, + 482, + 453, + 477, + 514, + 539, + 491, + 506, + 533, + 510, + 490, + 514, + 472, + 518, + 492, + 467, + 504, + 520, + 460, + 496, + 532, + 511, + 500, + 473, + 476, + 448, + 500, + 461, + 467, + 500, + 549, + 488, + 507, + 477, + 494, + 464, + 467, + 511, + 515, + 461, + 483, + 485, + 485, + 493, + 509, + 510, + 426, + 486, + 521, + 503, + 509, + 452, + 495, + 506, + 524, + 480, + 452, + 510, + 485, + 506, + 482, + 519, + 481, + 508, + 492, + 503, + 475, + 494, + 487, + 485, + 483, + 502, + 524, + 500, + 476, + 522, + 500, + 514, + 454, + 489, + 494, + 509, + 510, + 496, + 521, + 477, + 483, + 499, + 424, + 454, + 477, + 485, + 510, + 496, + 519, + 521, + 506, + 503, + 485, + 423, + 495, + 507, + 465, + 490, + 490, + 494, + 515, + 501, + 508, + 498, + 460, + 455, + 495, + 510, + 510, + 504, + 505, + 475, + 467, + 488, + 479, + 421, + 503, + 506, + 455, + 454, + 499, + 475, + 510, + 500, + 490, + 507, + 510, + 480, + 483, + 504, + 503, + 450, + 495, + 508, + 477, + 493, + 474, + 519, + 466, + 489, + 479, + 513, + 481, + 444, + 469, + 504, + 498, + 491, + 463, + 450, + 506, + 516, + 482, + 494, + 480, + 530, + 492, + 512, + 479, + 491, + 468, + 486, + 474, + 471, + 471, + 502, + 524, + 493, + 508, + 487, + 492, + 489, + 508, + 471, + 500, + 503, + 495, + 461, + 482, + 462, + 496, + 473, + 459, + 498, + 508, + 509, + 504, + 488, + 473, + 486, + 492, + 484, + 509, + 485, + 482, + 464, + 532, + 485, + 492, + 466, + 464, + 491, + 520, + 496, + 480, + 473, + 494, + 449, + 478, + 503, + 479, + 482, + 483, + 479, + 506, + 462, + 491, + 475, + 483, + 488, + 475, + 468, + 508, + 501, + 464, + 501, + 492, + 492, + 461, + 513, + 479, + 534, + 477, + 476, + 470, + 517, + 466, + 470, + 485, + 454, + 477, + 424, + 468, + 507, + 485, + 425, + 514, + 467, + 500, + 474, + 491, + 501, + 478, + 482, + 483, + 501, + 463, + 492, + 466, + 488, + 460, + 451, + 465, + 476, + 479, + 513, + 468, + 471, + 472, + 488, + 505, + 502, + 469, + 472, + 469, + 520, + 514, + 453, + 498, + 487, + 514, + 526, + 451, + 473, + 527, + 516, + 482, + 542, + 491, + 464, + 470, + 533, + 464, + 473, + 468, + 532, + 509, + 459, + 520, + 488, + 446, + 520, + 492, + 459, + 494, + 491, + 519, + 487, + 491, + 496, + 430, + 499, + 507, + 492, + 514, + 506, + 486, + 499, + 463, + 493, + 486, + 447, + 508, + 504, + 523, + 493, + 481, + 468, + 503, + 481, + 454, + 483, + 498, + 518, + 488, + 462, + 477, + 518, + 476, + 466, + 485, + 489, + 514, + 461, + 503, + 470, + 458, + 464, + 488, + 501, + 508, + 484, + 474, + 479, + 488, + 518, + 510, + 520, + 495, + 505, + 500, + 503, + 450, + 447, + 512, + 486, + 489, + 486, + 530, + 473, + 508, + 472, + 507, + 472, + 483, + 519, + 485, + 490, + 464, + 499, + 489, + 447, + 481, + 479, + 455, + 470, + 474, + 463, + 487, + 457, + 469, + 502, + 495, + 512, + 482, + 501, + 483, + 469, + 523, + 482, + 521, + 473, + 480, + 521, + 504, + 498, + 513, + 492, + 506, + 499, + 509, + 503, + 464, + 495, + 470, + 481, + 457, + 458, + 496, + 493, + 490, + 491, + 505, + 496, + 467, + 470, + 481, + 470, + 507, + 532, + 520, + 462, + 482, + 488, + 425, + 466, + 493, + 492, + 503, + 482, + 474, + 465, + 461, + 468, + 495, + 513, + 533, + 486, + 495, + 491, + 466, + 494, + 487, + 463, + 489, + 503, + 494, + 514, + 503, + 482, + 455, + 501, + 536, + 521, + 493, + 474, + 490, + 502, + 482, + 492, + 476, + 482, + 479, + 521, + 514, + 443, + 526, + 484, + 498, + 457, + 479, + 502, + 437, + 502, + 497, + 501, + 482, + 457, + 450, + 481, + 457, + 517, + 518, + 512, + 491, + 494, + 490, + 470, + 482, + 500, + 479, + 491, + 512, + 500, + 526, + 505, + 483, + 495, + 480, + 535, + 482, + 511, + 511, + 463, + 472, + 503, + 511, + 505, + 496, + 448, + 508, + 492, + 481, + 461, + 466, + 484, + 475, + 471, + 464, + 475, + 482, + 463, + 483, + 545, + 469, + 486, + 491, + 528, + 517, + 491, + 504, + 451, + 476, + 467, + 487, + 503, + 480, + 474, + 506, + 491, + 473, + 518, + 425, + 482, + 467, + 521, + 471, + 494, + 454, + 472, + 535, + 476, + 494, + 516, + 496, + 489, + 495, + 448, + 520, + 482, + 520, + 498, + 472, + 504, + 502, + 463, + 485, + 477, + 520, + 506, + 470, + 466, + 497, + 511, + 482, + 457, + 491, + 516, + 495, + 512, + 447, + 472, + 474, + 507, + 516, + 513, + 515, + 462, + 485, + 480, + 471, + 520, + 467, + 527, + 467, + 475, + 509, + 484, + 464, + 487, + 507, + 504, + 519, + 484, + 445, + 490, + 499, + 477, + 497, + 548, + 466, + 476, + 505, + 487, + 501, + 509, + 488, + 489, + 510, + 474, + 500, + 516, + 492, + 501, + 477, + 525, + 530, + 503, + 498, + 467, + 480, + 476, + 517, + 523, + 497, + 469, + 457, + 466, + 472, + 469, + 481, + 487, + 451, + 488, + 482, + 504, + 481, + 489, + 490, + 492, + 499, + 505, + 517, + 496, + 517, + 477, + 427, + 473, + 506, + 475, + 486, + 507, + 472, + 467, + 520, + 474, + 452, + 476, + 494, + 478, + 481, + 485, + 477, + 441, + 508, + 519, + 477, + 490, + 460, + 466, + 517, + 470, + 468, + 525, + 497, + 519, + 475, + 492, + 466, + 482, + 523, + 476, + 502, + 483, + 466, + 475, + 487, + 479, + 498, + 472, + 463, + 479, + 488, + 460, + 477, + 478, + 451, + 465, + 484, + 503, + 521, + 522, + 476, + 503, + 477, + 476, + 477, + 513, + 506, + 465, + 481, + 462, + 485, + 509, + 498, + 510, + 483, + 511, + 510, + 500, + 478, + 488, + 487, + 526, + 506, + 525, + 489, + 527, + 484, + 442, + 506, + 481, + 506, + 502, + 498, + 435, + 486, + 495, + 487, + 496, + 487, + 457, + 513, + 540, + 469, + 496, + 494, + 488, + 499, + 471, + 471, + 485, + 505, + 453, + 486, + 491, + 507, + 475, + 512, + 482, + 462, + 486, + 476, + 466, + 445, + 490, + 519, + 484, + 492, + 496, + 489, + 509, + 520, + 468, + 510, + 505, + 473, + 486, + 538, + 495, + 484, + 477, + 481, + 491, + 535, + 474, + 494, + 496, + 497, + 475, + 472, + 485, + 510, + 481, + 564, + 487, + 466, + 502, + 496, + 498, + 514, + 483, + 495, + 514, + 497, + 482, + 492, + 479, + 487, + 482, + 471, + 506, + 488, + 515, + 464, + 511, + 511, + 502, + 504, + 451, + 505, + 470, + 498, + 488, + 498, + 502, + 444, + 464, + 480, + 500, + 503, + 521, + 473, + 482, + 469, + 472, + 538, + 470, + 492, + 491, + 482, + 488, + 497, + 499, + 465, + 462, + 496, + 469, + 492, + 481, + 501, + 456, + 498, + 439, + 533, + 490, + 449, + 473, + 491, + 476, + 501, + 492, + 489, + 461, + 448, + 481, + 486, + 456, + 484, + 492, + 474, + 515, + 471, + 458, + 523, + 483, + 513, + 484, + 472, + 486, + 518, + 443, + 463, + 507, + 487, + 451, + 490, + 485, + 488, + 495, + 525, + 484, + 500, + 478, + 504, + 470, + 491, + 449, + 471, + 477, + 472, + 468, + 506, + 507, + 520, + 473, + 520, + 465, + 507, + 480, + 475, + 491, + 487, + 495, + 473, + 440, + 495, + 473, + 433, + 505, + 506, + 516, + 466, + 462, + 505, + 472, + 482, + 494, + 512, + 497, + 476, + 515, + 466, + 491, + 466, + 511, + 477, + 496, + 535, + 513, + 458, + 509, + 477, + 522, + 485, + 471, + 440, + 504, + 480, + 479, + 485, + 474, + 485, + 490, + 518, + 507, + 508, + 511, + 454, + 475, + 502, + 480, + 476, + 490, + 507, + 485, + 484, + 494, + 530, + 486, + 484, + 523, + 458, + 476, + 473, + 452, + 472, + 461, + 474, + 500, + 518, + 487, + 508, + 505, + 474, + 484, + 532, + 462, + 462, + 472, + 445, + 489, + 508, + 517, + 474, + 484, + 483, + 456, + 482, + 488, + 452, + 511, + 449, + 501, + 489, + 454, + 477, + 537, + 484, + 485, + 516, + 463, + 507, + 500, + 479, + 480, + 483, + 491, + 473, + 443, + 483, + 482, + 497, + 486, + 502, + 497, + 530, + 515, + 552, + 477, + 517, + 478, + 513, + 490, + 536, + 501, + 486, + 503, + 499, + 502, + 514, + 521, + 493, + 514, + 471, + 459, + 503, + 466, + 465, + 516, + 532, + 496, + 495, + 513, + 469, + 476, + 476, + 501, + 501, + 491, + 479, + 469, + 463, + 468, + 505, + 520, + 509, + 474, + 489, + 483, + 504, + 494, + 497, + 493, + 506, + 516, + 503, + 472, + 456, + 518, + 467, + 458, + 487, + 458, + 493, + 525, + 480, + 453, + 485, + 484, + 491, + 493, + 478, + 482, + 529, + 484, + 486, + 471, + 465, + 481, + 498, + 511, + 530, + 486, + 464, + 461, + 470, + 489, + 516, + 507, + 495, + 493, + 492, + 445, + 483, + 517, + 518, + 460, + 481, + 478, + 498, + 480, + 494, + 505, + 516, + 460, + 448, + 477, + 460, + 490, + 490, + 465, + 496, + 535, + 476, + 507, + 468, + 480, + 508, + 505, + 534, + 537, + 464, + 502, + 481, + 497, + 481, + 477, + 487, + 467, + 483, + 491, + 512, + 479, + 489, + 507, + 484, + 478, + 451, + 472, + 470, + 509, + 486, + 464, + 478, + 481, + 520, + 492, + 480, + 459, + 489, + 437, + 495, + 479, + 489, + 486, + 474, + 458, + 470, + 479, + 464, + 481, + 468, + 516, + 479, + 509, + 501, + 477, + 523, + 481, + 487, + 507, + 517, + 487, + 518, + 508, + 518, + 484, + 496, + 487, + 507, + 479, + 460, + 491, + 460, + 523, + 520, + 471, + 505, + 490, + 451, + 468, + 494, + 482, + 491, + 462, + 490, + 434, + 485, + 496, + 466, + 482, + 490, + 480, + 452, + 487, + 471, + 468, + 481, + 538, + 475, + 482, + 495, + 491, + 489, + 493, + 469, + 483, + 480, + 508, + 516, + 473, + 487, + 513, + 493, + 506, + 489, + 478, + 480, + 469, + 455, + 487, + 509, + 473, + 490, + 512, + 510, + 488, + 517 + ] + }, + { + "counters": [ + 492, + 496, + 514, + 481, + 463, + 481, + 487, + 491, + 438, + 495, + 531, + 489, + 445, + 508, + 527, + 577, + 468, + 537, + 474, + 503, + 439, + 514, + 541, + 515, + 463, + 470, + 483, + 466, + 484, + 522, + 492, + 468, + 497, + 523, + 500, + 492, + 472, + 503, + 506, + 469, + 496, + 509, + 502, + 439, + 476, + 508, + 483, + 512, + 502, + 463, + 488, + 453, + 482, + 503, + 542, + 524, + 483, + 461, + 478, + 482, + 496, + 503, + 510, + 455, + 513, + 509, + 437, + 467, + 483, + 483, + 458, + 539, + 503, + 476, + 491, + 504, + 520, + 474, + 485, + 507, + 485, + 524, + 507, + 459, + 485, + 531, + 497, + 474, + 469, + 499, + 506, + 472, + 491, + 486, + 497, + 525, + 501, + 491, + 510, + 487, + 471, + 490, + 478, + 486, + 493, + 471, + 449, + 474, + 494, + 452, + 509, + 484, + 473, + 493, + 489, + 510, + 454, + 479, + 482, + 471, + 505, + 485, + 472, + 522, + 493, + 459, + 499, + 517, + 480, + 487, + 459, + 488, + 482, + 460, + 491, + 475, + 476, + 506, + 466, + 491, + 511, + 477, + 485, + 469, + 511, + 486, + 514, + 513, + 465, + 531, + 464, + 483, + 488, + 468, + 484, + 499, + 487, + 502, + 475, + 492, + 495, + 510, + 493, + 486, + 513, + 488, + 419, + 486, + 520, + 528, + 493, + 535, + 443, + 463, + 429, + 548, + 467, + 452, + 535, + 485, + 457, + 436, + 473, + 510, + 484, + 479, + 505, + 548, + 486, + 526, + 518, + 493, + 532, + 483, + 471, + 491, + 491, + 507, + 458, + 482, + 515, + 489, + 450, + 502, + 457, + 467, + 474, + 488, + 524, + 469, + 474, + 510, + 489, + 470, + 493, + 494, + 451, + 509, + 522, + 466, + 484, + 467, + 524, + 483, + 502, + 513, + 464, + 536, + 477, + 489, + 479, + 457, + 516, + 482, + 524, + 460, + 439, + 509, + 508, + 511, + 487, + 500, + 539, + 505, + 485, + 477, + 535, + 464, + 495, + 497, + 526, + 468, + 524, + 488, + 459, + 482, + 507, + 476, + 498, + 454, + 506, + 457, + 485, + 510, + 488, + 511, + 502, + 513, + 503, + 511, + 518, + 507, + 470, + 513, + 476, + 471, + 520, + 495, + 504, + 490, + 492, + 489, + 467, + 480, + 463, + 489, + 499, + 479, + 507, + 510, + 529, + 443, + 504, + 483, + 484, + 452, + 459, + 486, + 481, + 475, + 501, + 474, + 485, + 529, + 486, + 469, + 451, + 484, + 499, + 484, + 483, + 504, + 507, + 471, + 517, + 508, + 497, + 444, + 484, + 474, + 508, + 508, + 518, + 489, + 479, + 498, + 484, + 502, + 485, + 537, + 464, + 458, + 471, + 500, + 494, + 486, + 513, + 490, + 521, + 450, + 482, + 473, + 479, + 478, + 461, + 485, + 480, + 489, + 471, + 521, + 524, + 482, + 514, + 477, + 550, + 486, + 532, + 514, + 497, + 499, + 491, + 489, + 500, + 496, + 484, + 492, + 460, + 480, + 483, + 482, + 512, + 460, + 453, + 515, + 477, + 478, + 502, + 524, + 498, + 494, + 525, + 481, + 458, + 493, + 473, + 511, + 469, + 477, + 495, + 445, + 476, + 494, + 504, + 488, + 481, + 480, + 479, + 502, + 482, + 512, + 485, + 502, + 501, + 467, + 448, + 497, + 509, + 507, + 471, + 478, + 483, + 504, + 513, + 436, + 440, + 507, + 491, + 504, + 507, + 466, + 488, + 480, + 506, + 479, + 486, + 482, + 480, + 497, + 502, + 457, + 489, + 470, + 507, + 480, + 481, + 458, + 502, + 485, + 498, + 461, + 531, + 547, + 464, + 465, + 510, + 461, + 461, + 474, + 529, + 499, + 503, + 477, + 537, + 509, + 481, + 539, + 528, + 508, + 475, + 476, + 487, + 504, + 495, + 484, + 486, + 501, + 462, + 501, + 465, + 506, + 478, + 472, + 545, + 487, + 464, + 463, + 516, + 474, + 520, + 462, + 502, + 500, + 453, + 477, + 476, + 477, + 504, + 482, + 488, + 506, + 472, + 500, + 500, + 507, + 497, + 498, + 465, + 516, + 457, + 474, + 464, + 451, + 522, + 484, + 429, + 508, + 449, + 467, + 488, + 503, + 483, + 488, + 492, + 434, + 472, + 522, + 486, + 520, + 507, + 517, + 499, + 481, + 445, + 465, + 516, + 455, + 520, + 438, + 467, + 489, + 498, + 500, + 487, + 505, + 477, + 487, + 480, + 491, + 515, + 496, + 462, + 456, + 479, + 498, + 494, + 478, + 476, + 476, + 478, + 467, + 530, + 477, + 477, + 495, + 487, + 523, + 544, + 473, + 496, + 493, + 465, + 502, + 497, + 494, + 496, + 479, + 485, + 473, + 473, + 519, + 477, + 487, + 511, + 468, + 478, + 491, + 529, + 473, + 523, + 498, + 474, + 455, + 482, + 509, + 500, + 491, + 501, + 481, + 500, + 465, + 477, + 471, + 513, + 514, + 489, + 512, + 537, + 487, + 506, + 484, + 446, + 480, + 487, + 511, + 482, + 467, + 497, + 484, + 474, + 496, + 477, + 522, + 510, + 521, + 480, + 461, + 531, + 478, + 465, + 470, + 479, + 455, + 533, + 452, + 486, + 499, + 479, + 459, + 494, + 434, + 485, + 466, + 479, + 481, + 510, + 493, + 477, + 479, + 468, + 491, + 487, + 477, + 484, + 491, + 494, + 431, + 534, + 493, + 498, + 477, + 476, + 448, + 445, + 488, + 453, + 484, + 454, + 487, + 446, + 460, + 500, + 468, + 479, + 483, + 498, + 457, + 449, + 474, + 481, + 503, + 497, + 492, + 483, + 532, + 482, + 482, + 500, + 453, + 499, + 459, + 506, + 492, + 524, + 491, + 467, + 495, + 510, + 471, + 491, + 503, + 460, + 484, + 448, + 476, + 513, + 459, + 521, + 476, + 484, + 491, + 494, + 566, + 479, + 555, + 516, + 493, + 482, + 520, + 510, + 470, + 460, + 497, + 478, + 480, + 515, + 496, + 496, + 503, + 485, + 503, + 508, + 476, + 516, + 510, + 510, + 518, + 486, + 487, + 460, + 492, + 495, + 499, + 470, + 461, + 507, + 510, + 494, + 488, + 503, + 505, + 550, + 514, + 494, + 492, + 488, + 505, + 452, + 488, + 474, + 479, + 466, + 463, + 457, + 489, + 510, + 449, + 490, + 444, + 483, + 483, + 509, + 501, + 512, + 465, + 476, + 442, + 487, + 502, + 458, + 524, + 461, + 500, + 482, + 492, + 469, + 460, + 508, + 459, + 452, + 487, + 481, + 466, + 515, + 491, + 489, + 482, + 489, + 486, + 470, + 505, + 494, + 544, + 509, + 503, + 515, + 502, + 498, + 461, + 530, + 460, + 494, + 463, + 470, + 551, + 478, + 483, + 535, + 506, + 511, + 491, + 513, + 506, + 461, + 484, + 430, + 462, + 494, + 521, + 458, + 463, + 503, + 506, + 446, + 513, + 479, + 491, + 470, + 491, + 467, + 497, + 475, + 473, + 511, + 458, + 502, + 489, + 455, + 487, + 475, + 466, + 474, + 518, + 493, + 511, + 503, + 449, + 467, + 444, + 529, + 494, + 485, + 509, + 516, + 463, + 501, + 472, + 468, + 491, + 526, + 511, + 461, + 486, + 497, + 525, + 469, + 478, + 528, + 482, + 462, + 464, + 524, + 505, + 435, + 474, + 441, + 484, + 532, + 496, + 462, + 480, + 482, + 526, + 504, + 466, + 511, + 500, + 470, + 502, + 419, + 500, + 446, + 467, + 503, + 471, + 471, + 490, + 517, + 482, + 505, + 493, + 502, + 488, + 491, + 469, + 506, + 440, + 483, + 487, + 471, + 493, + 475, + 502, + 467, + 494, + 471, + 472, + 513, + 506, + 512, + 523, + 503, + 469, + 498, + 498, + 470, + 481, + 451, + 488, + 493, + 487, + 511, + 474, + 494, + 506, + 513, + 531, + 518, + 492, + 487, + 470, + 496, + 485, + 477, + 481, + 520, + 512, + 465, + 479, + 494, + 545, + 473, + 483, + 475, + 515, + 475, + 476, + 465, + 498, + 497, + 497, + 445, + 508, + 492, + 525, + 499, + 484, + 479, + 484, + 434, + 505, + 488, + 493, + 508, + 484, + 475, + 475, + 501, + 510, + 498, + 483, + 488, + 464, + 502, + 471, + 494, + 476, + 504, + 458, + 444, + 485, + 478, + 454, + 494, + 472, + 494, + 488, + 483, + 497, + 489, + 484, + 513, + 520, + 483, + 476, + 464, + 518, + 500, + 502, + 518, + 495, + 456, + 487, + 483, + 473, + 496, + 493, + 490, + 508, + 465, + 497, + 475, + 522, + 475, + 483, + 492, + 455, + 470, + 489, + 525, + 480, + 473, + 503, + 504, + 477, + 473, + 529, + 500, + 474, + 501, + 512, + 498, + 510, + 495, + 497, + 489, + 474, + 487, + 489, + 504, + 466, + 503, + 472, + 465, + 510, + 476, + 442, + 474, + 507, + 495, + 490, + 483, + 466, + 489, + 472, + 466, + 519, + 471, + 500, + 461, + 494, + 500, + 510, + 496, + 470, + 485, + 466, + 484, + 516, + 493, + 500, + 471, + 489, + 454, + 494, + 465, + 530, + 536, + 491, + 478, + 491, + 443, + 434, + 468, + 502, + 477, + 511, + 479, + 500, + 499, + 451, + 503, + 493, + 501, + 485, + 450, + 482, + 482, + 438, + 516, + 490, + 508, + 500, + 491, + 462, + 533, + 478, + 463, + 517, + 497, + 466, + 493, + 495, + 489, + 475, + 478, + 509, + 450, + 506, + 504, + 481, + 464, + 536, + 534, + 466, + 463, + 510, + 507, + 482, + 503, + 478, + 469, + 465, + 513, + 481, + 446, + 510, + 484, + 443, + 516, + 482, + 471, + 505, + 471, + 485, + 491, + 471, + 496, + 470, + 509, + 435, + 487, + 502, + 475, + 496, + 519, + 448, + 477, + 481, + 492, + 509, + 458, + 511, + 483, + 465, + 477, + 493, + 487, + 513, + 488, + 501, + 501, + 501, + 490, + 479, + 505, + 526, + 496, + 548, + 486, + 517, + 424, + 498, + 490, + 512, + 476, + 454, + 476, + 506, + 473, + 466, + 490, + 448, + 498, + 494, + 512, + 476, + 490, + 477, + 460, + 473, + 511, + 489, + 510, + 487, + 454, + 472, + 485, + 478, + 476, + 459, + 501, + 506, + 472, + 478, + 505, + 507, + 486, + 487, + 498, + 511, + 504, + 475, + 466, + 503, + 496, + 473, + 500, + 450, + 485, + 477, + 486, + 507, + 498, + 486, + 514, + 466, + 471, + 454, + 455, + 488, + 503, + 504, + 504, + 527, + 493, + 515, + 478, + 486, + 496, + 508, + 482, + 511, + 475, + 470, + 485, + 469, + 493, + 498, + 492, + 500, + 528, + 496, + 497, + 473, + 486, + 488, + 474, + 470, + 484, + 458, + 500, + 510, + 477, + 503, + 479, + 471, + 488, + 510, + 504, + 436, + 491, + 486, + 461, + 477, + 483, + 452, + 465, + 474, + 510, + 486, + 531, + 508, + 499, + 451, + 497, + 504, + 490, + 469, + 452, + 506, + 459, + 487, + 481, + 510, + 469, + 486, + 517, + 488, + 463, + 480, + 534, + 483, + 513, + 468, + 471, + 461, + 531, + 488, + 466, + 490, + 513, + 513, + 507, + 492, + 493, + 479, + 505, + 476, + 485, + 500, + 485, + 503, + 453, + 478, + 500, + 471, + 473, + 466, + 465, + 532, + 513, + 517, + 518, + 532, + 501, + 491, + 451, + 482, + 472, + 483, + 474, + 529, + 472, + 512, + 482, + 518, + 468, + 435, + 460, + 497, + 538, + 484, + 500, + 446, + 494, + 505, + 498, + 505, + 487, + 517, + 496, + 489, + 475, + 433, + 452, + 486, + 483, + 498, + 476, + 510, + 496, + 517, + 490, + 465, + 462, + 463, + 462, + 465, + 486, + 504, + 458, + 489, + 517, + 474, + 514, + 536, + 527, + 446, + 536, + 493, + 492, + 490, + 469, + 483, + 512, + 449, + 536, + 501, + 499, + 477, + 489, + 483, + 518, + 484, + 484, + 485, + 483, + 470, + 481, + 500, + 474, + 469, + 493, + 461, + 506, + 468, + 496, + 483, + 466, + 499, + 521, + 466, + 510, + 489, + 519, + 523, + 470, + 486, + 489, + 536, + 499, + 501, + 489, + 501, + 473, + 485, + 491, + 514, + 508, + 495, + 457, + 479, + 484, + 474, + 496, + 481, + 515, + 468, + 480, + 482, + 519, + 495, + 496, + 511, + 464, + 511, + 483, + 441, + 494, + 486, + 512, + 434, + 501, + 509, + 430, + 473, + 529, + 487, + 485, + 512, + 460, + 513, + 436, + 487, + 493, + 499, + 491, + 469, + 478, + 491, + 499, + 469, + 484, + 459, + 442, + 441, + 496, + 481, + 540, + 506, + 470, + 516, + 507, + 466, + 461, + 534, + 514, + 469, + 516, + 453, + 436, + 468, + 478, + 468, + 454, + 464, + 505, + 510, + 532, + 497, + 495, + 504, + 482, + 500, + 497, + 488, + 470, + 502, + 490, + 502, + 466, + 461, + 494, + 491, + 511, + 531, + 465, + 532, + 474, + 499, + 496, + 481, + 486, + 473, + 496, + 494, + 501, + 495, + 477, + 509, + 525, + 510, + 474, + 514, + 493, + 510, + 513, + 505, + 475, + 493, + 458, + 462, + 467, + 468, + 474, + 501, + 539, + 509, + 480, + 500, + 480, + 468, + 464, + 500, + 457, + 497, + 467, + 472, + 528, + 473, + 472, + 528, + 488, + 482, + 490, + 502, + 523, + 469, + 475, + 493, + 473, + 455, + 461, + 469, + 480, + 460, + 497, + 478, + 509, + 495, + 491, + 523, + 508, + 495, + 493, + 460, + 460, + 495, + 483, + 498, + 494, + 501, + 471, + 495, + 465, + 476, + 476, + 502, + 495, + 503, + 491, + 454, + 458, + 511, + 497, + 541, + 468, + 460, + 499, + 495, + 481, + 467, + 453, + 512, + 451, + 502, + 481, + 495, + 456, + 481, + 484, + 459, + 520, + 470, + 480, + 512, + 510, + 472, + 446, + 469, + 496, + 488, + 493, + 503, + 513, + 514, + 500, + 495, + 501, + 484, + 466, + 504, + 494, + 451, + 477, + 494, + 494, + 471, + 481, + 514, + 458, + 488, + 507, + 489, + 465, + 475, + 478, + 496, + 462, + 465, + 499, + 488, + 502, + 507, + 448, + 460, + 480, + 486, + 466, + 500, + 528, + 486, + 475, + 475, + 493, + 458, + 475, + 488, + 513, + 493, + 468, + 508, + 454, + 533, + 474, + 445, + 481, + 492, + 522, + 506, + 514, + 496, + 470, + 509, + 504, + 512, + 498, + 519, + 531, + 499, + 474, + 494, + 497, + 515, + 506, + 506, + 515, + 497, + 472, + 501, + 514, + 490, + 468, + 483, + 491, + 532, + 494, + 533, + 506, + 491, + 446, + 497, + 469, + 478, + 478, + 469, + 467, + 496, + 518, + 456, + 504, + 472, + 469, + 486, + 486, + 506, + 440, + 473, + 475, + 459, + 496, + 506, + 517, + 483, + 484, + 507, + 481, + 467, + 483, + 481, + 504, + 446, + 467, + 476, + 477, + 447, + 489, + 494, + 474, + 511, + 500, + 486, + 509, + 499, + 488, + 480, + 503, + 484, + 485, + 503, + 483, + 483, + 483, + 447, + 491, + 502, + 492, + 494, + 491, + 465, + 503, + 499, + 484, + 461, + 497, + 482, + 477, + 468, + 511, + 481, + 485, + 470, + 490, + 515, + 526, + 457, + 510, + 476, + 501, + 437, + 496, + 531, + 495, + 472, + 485, + 471, + 517, + 518, + 519, + 459, + 490, + 522, + 476, + 465, + 466, + 508, + 477, + 482, + 503, + 476, + 486, + 473, + 499, + 493, + 490, + 492, + 478, + 522, + 523, + 494, + 512, + 482, + 502, + 474, + 474, + 482, + 486, + 502, + 462, + 536, + 486, + 484, + 474, + 503, + 506, + 508, + 506, + 500, + 468, + 494, + 462, + 528, + 451, + 501, + 457, + 476, + 481, + 493, + 479, + 467, + 498, + 508, + 520, + 492, + 447, + 508, + 491, + 490, + 448, + 466, + 474, + 497, + 469, + 521, + 538, + 501, + 501, + 429, + 510, + 499, + 509, + 487, + 493, + 518, + 495, + 482, + 494, + 502, + 529, + 460, + 553, + 506, + 490, + 534, + 453, + 500, + 491, + 521, + 457, + 452, + 520, + 517, + 456, + 476, + 474, + 495, + 456, + 496, + 516, + 490, + 485, + 463, + 487, + 453, + 466, + 486, + 459, + 485, + 484, + 491, + 463, + 499, + 511, + 454, + 488, + 461, + 507, + 493, + 507, + 499, + 505, + 461, + 496, + 546, + 492, + 500, + 437, + 493, + 495, + 484, + 453, + 465, + 480, + 477, + 508, + 450, + 475, + 461, + 480, + 486, + 491, + 476, + 534, + 515, + 475, + 486, + 539, + 462, + 445, + 529, + 490, + 470, + 463, + 520, + 493, + 496, + 469, + 487, + 488, + 519, + 498, + 534, + 522, + 470, + 494, + 494, + 497, + 537, + 498, + 484, + 468, + 475, + 503, + 515, + 450, + 460, + 479, + 487, + 467, + 466, + 460, + 504, + 492, + 516, + 503, + 465, + 480, + 487, + 477, + 484, + 473, + 470, + 526, + 471, + 460, + 500, + 479, + 484, + 483, + 490, + 522, + 475, + 500, + 486, + 490, + 485, + 434, + 480 + ] + }, + { + "counters": [ + 489, + 519, + 466, + 483, + 484, + 514, + 505, + 495, + 485, + 472, + 525, + 511, + 497, + 486, + 478, + 481, + 483, + 502, + 468, + 473, + 546, + 482, + 497, + 456, + 471, + 501, + 479, + 480, + 489, + 513, + 499, + 481, + 507, + 472, + 503, + 467, + 503, + 535, + 502, + 492, + 518, + 487, + 482, + 469, + 515, + 488, + 475, + 512, + 514, + 491, + 478, + 485, + 478, + 476, + 499, + 447, + 491, + 534, + 493, + 491, + 471, + 465, + 500, + 455, + 505, + 484, + 461, + 480, + 497, + 463, + 497, + 488, + 467, + 507, + 504, + 465, + 503, + 492, + 503, + 486, + 497, + 494, + 527, + 485, + 478, + 514, + 471, + 477, + 493, + 502, + 493, + 489, + 505, + 504, + 483, + 508, + 473, + 503, + 520, + 520, + 460, + 467, + 509, + 490, + 467, + 466, + 497, + 484, + 490, + 494, + 498, + 453, + 454, + 469, + 529, + 462, + 491, + 513, + 499, + 520, + 456, + 494, + 507, + 515, + 504, + 474, + 489, + 495, + 483, + 467, + 468, + 479, + 510, + 488, + 461, + 484, + 482, + 492, + 482, + 468, + 497, + 554, + 503, + 457, + 496, + 483, + 464, + 447, + 446, + 518, + 449, + 531, + 511, + 494, + 497, + 512, + 483, + 486, + 476, + 499, + 506, + 481, + 511, + 505, + 518, + 506, + 506, + 508, + 501, + 502, + 511, + 493, + 464, + 515, + 488, + 494, + 488, + 513, + 503, + 440, + 451, + 495, + 451, + 437, + 516, + 495, + 524, + 493, + 485, + 490, + 464, + 504, + 456, + 481, + 517, + 486, + 485, + 471, + 506, + 492, + 531, + 468, + 529, + 495, + 486, + 512, + 494, + 484, + 488, + 476, + 486, + 499, + 504, + 452, + 488, + 469, + 467, + 479, + 474, + 446, + 473, + 473, + 465, + 501, + 487, + 499, + 488, + 487, + 502, + 514, + 469, + 483, + 487, + 469, + 490, + 486, + 446, + 445, + 509, + 480, + 500, + 478, + 490, + 496, + 514, + 464, + 512, + 468, + 480, + 503, + 530, + 494, + 488, + 493, + 535, + 508, + 483, + 475, + 488, + 478, + 483, + 491, + 455, + 509, + 473, + 500, + 514, + 458, + 488, + 473, + 524, + 496, + 477, + 477, + 497, + 512, + 476, + 435, + 474, + 497, + 513, + 500, + 496, + 475, + 464, + 514, + 496, + 528, + 501, + 476, + 489, + 456, + 455, + 481, + 498, + 456, + 489, + 455, + 469, + 477, + 466, + 491, + 485, + 466, + 490, + 462, + 456, + 470, + 479, + 494, + 495, + 514, + 514, + 495, + 478, + 457, + 456, + 472, + 496, + 514, + 501, + 500, + 486, + 508, + 528, + 472, + 488, + 475, + 475, + 458, + 437, + 502, + 468, + 464, + 498, + 511, + 496, + 504, + 457, + 489, + 494, + 478, + 466, + 479, + 482, + 497, + 497, + 472, + 456, + 540, + 489, + 484, + 470, + 514, + 498, + 492, + 493, + 481, + 471, + 477, + 477, + 472, + 514, + 498, + 496, + 473, + 498, + 510, + 528, + 502, + 467, + 502, + 492, + 490, + 465, + 513, + 504, + 500, + 498, + 508, + 501, + 499, + 499, + 501, + 459, + 491, + 491, + 494, + 504, + 464, + 506, + 468, + 499, + 499, + 492, + 474, + 484, + 464, + 492, + 451, + 490, + 500, + 465, + 472, + 526, + 447, + 498, + 485, + 495, + 489, + 505, + 479, + 452, + 470, + 512, + 496, + 489, + 486, + 474, + 510, + 477, + 468, + 467, + 496, + 448, + 469, + 506, + 443, + 505, + 488, + 461, + 493, + 464, + 510, + 504, + 476, + 504, + 520, + 468, + 495, + 500, + 499, + 487, + 469, + 481, + 480, + 468, + 500, + 455, + 479, + 449, + 459, + 452, + 485, + 462, + 533, + 487, + 493, + 501, + 513, + 473, + 473, + 486, + 463, + 499, + 463, + 463, + 463, + 513, + 474, + 508, + 533, + 459, + 494, + 496, + 455, + 490, + 490, + 480, + 477, + 474, + 463, + 508, + 493, + 502, + 492, + 493, + 490, + 510, + 497, + 480, + 502, + 456, + 479, + 458, + 477, + 499, + 500, + 457, + 451, + 459, + 475, + 495, + 471, + 482, + 472, + 473, + 489, + 488, + 445, + 462, + 504, + 496, + 453, + 503, + 484, + 488, + 491, + 501, + 481, + 477, + 485, + 492, + 507, + 469, + 495, + 525, + 532, + 452, + 482, + 520, + 504, + 456, + 509, + 471, + 503, + 504, + 476, + 488, + 445, + 471, + 501, + 477, + 502, + 503, + 481, + 487, + 465, + 459, + 500, + 459, + 493, + 504, + 503, + 499, + 500, + 472, + 547, + 503, + 463, + 463, + 489, + 482, + 534, + 487, + 484, + 456, + 502, + 481, + 507, + 482, + 502, + 491, + 473, + 477, + 477, + 471, + 516, + 471, + 496, + 469, + 479, + 500, + 473, + 479, + 492, + 500, + 463, + 487, + 499, + 481, + 494, + 489, + 497, + 469, + 457, + 493, + 462, + 488, + 493, + 527, + 508, + 502, + 462, + 473, + 457, + 453, + 498, + 483, + 475, + 466, + 480, + 500, + 499, + 491, + 453, + 487, + 458, + 477, + 504, + 492, + 521, + 473, + 485, + 424, + 471, + 472, + 445, + 491, + 488, + 484, + 435, + 490, + 513, + 486, + 476, + 499, + 495, + 446, + 509, + 481, + 473, + 488, + 490, + 488, + 482, + 506, + 481, + 497, + 503, + 519, + 522, + 486, + 518, + 480, + 492, + 507, + 479, + 524, + 512, + 498, + 511, + 490, + 527, + 519, + 475, + 469, + 537, + 448, + 466, + 535, + 481, + 522, + 514, + 485, + 488, + 492, + 508, + 485, + 497, + 481, + 495, + 502, + 472, + 487, + 448, + 464, + 485, + 566, + 514, + 430, + 491, + 469, + 425, + 512, + 497, + 528, + 498, + 466, + 477, + 505, + 483, + 498, + 477, + 455, + 491, + 526, + 468, + 464, + 504, + 502, + 491, + 488, + 482, + 477, + 452, + 432, + 459, + 521, + 422, + 513, + 488, + 470, + 519, + 522, + 471, + 503, + 496, + 502, + 496, + 548, + 497, + 504, + 486, + 488, + 505, + 511, + 474, + 466, + 489, + 457, + 492, + 509, + 483, + 509, + 464, + 478, + 492, + 511, + 476, + 458, + 465, + 485, + 482, + 463, + 485, + 492, + 470, + 479, + 472, + 463, + 512, + 484, + 498, + 487, + 485, + 468, + 493, + 455, + 483, + 474, + 497, + 484, + 452, + 472, + 515, + 461, + 460, + 503, + 503, + 449, + 479, + 435, + 525, + 478, + 453, + 472, + 462, + 484, + 492, + 475, + 525, + 498, + 432, + 488, + 461, + 483, + 502, + 530, + 497, + 477, + 483, + 497, + 495, + 478, + 490, + 471, + 476, + 507, + 500, + 466, + 511, + 484, + 513, + 487, + 514, + 466, + 478, + 510, + 505, + 505, + 502, + 469, + 515, + 495, + 506, + 493, + 511, + 469, + 472, + 512, + 471, + 486, + 490, + 492, + 468, + 515, + 531, + 489, + 516, + 468, + 515, + 488, + 462, + 470, + 456, + 522, + 467, + 457, + 479, + 478, + 521, + 519, + 520, + 436, + 427, + 503, + 514, + 518, + 512, + 492, + 454, + 489, + 492, + 506, + 499, + 502, + 470, + 468, + 456, + 498, + 448, + 502, + 517, + 509, + 482, + 519, + 525, + 501, + 514, + 478, + 485, + 504, + 502, + 481, + 484, + 525, + 502, + 497, + 497, + 485, + 492, + 452, + 421, + 449, + 502, + 517, + 474, + 479, + 478, + 493, + 483, + 459, + 441, + 465, + 509, + 473, + 455, + 518, + 433, + 473, + 507, + 498, + 472, + 489, + 504, + 482, + 490, + 491, + 533, + 487, + 480, + 504, + 526, + 519, + 496, + 468, + 466, + 493, + 518, + 528, + 501, + 480, + 479, + 472, + 481, + 480, + 507, + 457, + 519, + 507, + 468, + 518, + 464, + 492, + 478, + 473, + 539, + 466, + 515, + 465, + 489, + 461, + 476, + 457, + 482, + 491, + 491, + 474, + 464, + 478, + 519, + 487, + 503, + 482, + 474, + 468, + 459, + 432, + 474, + 469, + 493, + 485, + 504, + 486, + 506, + 492, + 489, + 489, + 530, + 503, + 499, + 509, + 499, + 488, + 514, + 440, + 545, + 487, + 452, + 500, + 509, + 485, + 517, + 496, + 513, + 513, + 483, + 494, + 514, + 474, + 502, + 509, + 524, + 520, + 444, + 475, + 488, + 505, + 493, + 489, + 474, + 451, + 487, + 490, + 499, + 481, + 494, + 465, + 454, + 479, + 487, + 486, + 477, + 495, + 501, + 471, + 479, + 469, + 464, + 502, + 486, + 439, + 478, + 501, + 505, + 444, + 505, + 481, + 514, + 494, + 463, + 472, + 473, + 472, + 452, + 488, + 526, + 494, + 476, + 522, + 514, + 502, + 479, + 487, + 496, + 483, + 481, + 481, + 476, + 514, + 511, + 484, + 457, + 502, + 539, + 489, + 464, + 506, + 465, + 446, + 528, + 459, + 455, + 478, + 520, + 472, + 440, + 490, + 533, + 487, + 494, + 492, + 488, + 526, + 521, + 492, + 485, + 481, + 439, + 459, + 495, + 472, + 477, + 490, + 511, + 452, + 511, + 480, + 496, + 527, + 483, + 479, + 479, + 476, + 481, + 462, + 515, + 440, + 487, + 498, + 484, + 468, + 475, + 518, + 486, + 508, + 499, + 465, + 487, + 497, + 467, + 484, + 533, + 505, + 491, + 520, + 459, + 495, + 527, + 463, + 480, + 473, + 542, + 503, + 441, + 472, + 462, + 486, + 496, + 516, + 462, + 537, + 511, + 499, + 493, + 495, + 447, + 489, + 517, + 435, + 476, + 492, + 475, + 485, + 493, + 471, + 496, + 514, + 492, + 477, + 500, + 518, + 488, + 539, + 468, + 536, + 471, + 491, + 488, + 507, + 511, + 496, + 450, + 486, + 520, + 514, + 474, + 477, + 464, + 472, + 474, + 464, + 496, + 462, + 457, + 439, + 510, + 462, + 469, + 508, + 459, + 494, + 509, + 466, + 511, + 523, + 466, + 495, + 454, + 512, + 469, + 467, + 478, + 453, + 520, + 491, + 483, + 493, + 500, + 444, + 492, + 473, + 502, + 455, + 494, + 549, + 508, + 466, + 494, + 479, + 508, + 473, + 463, + 483, + 471, + 525, + 517, + 479, + 500, + 500, + 473, + 478, + 496, + 444, + 500, + 495, + 513, + 507, + 491, + 453, + 498, + 464, + 476, + 523, + 460, + 492, + 525, + 517, + 483, + 480, + 485, + 465, + 512, + 490, + 506, + 547, + 486, + 462, + 484, + 524, + 476, + 471, + 479, + 496, + 546, + 489, + 473, + 468, + 492, + 490, + 494, + 481, + 488, + 457, + 520, + 495, + 492, + 476, + 471, + 494, + 518, + 519, + 502, + 510, + 499, + 507, + 477, + 500, + 491, + 492, + 500, + 496, + 490, + 519, + 474, + 434, + 482, + 482, + 501, + 486, + 456, + 510, + 502, + 488, + 503, + 473, + 521, + 517, + 464, + 506, + 462, + 471, + 497, + 472, + 495, + 507, + 481, + 478, + 477, + 421, + 528, + 509, + 534, + 511, + 513, + 487, + 536, + 502, + 532, + 512, + 461, + 480, + 499, + 456, + 504, + 478, + 496, + 485, + 467, + 507, + 491, + 482, + 483, + 479, + 478, + 523, + 469, + 516, + 497, + 500, + 478, + 461, + 503, + 480, + 480, + 468, + 507, + 473, + 500, + 503, + 488, + 486, + 497, + 481, + 523, + 482, + 488, + 489, + 517, + 493, + 528, + 489, + 481, + 510, + 470, + 467, + 490, + 476, + 489, + 465, + 482, + 487, + 498, + 497, + 515, + 470, + 487, + 472, + 522, + 468, + 495, + 477, + 438, + 474, + 532, + 525, + 452, + 495, + 474, + 496, + 524, + 473, + 499, + 519, + 505, + 507, + 484, + 434, + 484, + 490, + 537, + 519, + 462, + 484, + 479, + 457, + 460, + 512, + 487, + 503, + 502, + 463, + 489, + 497, + 494, + 506, + 495, + 518, + 473, + 446, + 486, + 462, + 515, + 473, + 502, + 514, + 518, + 476, + 498, + 455, + 517, + 492, + 424, + 490, + 485, + 502, + 499, + 479, + 468, + 460, + 450, + 464, + 482, + 478, + 494, + 530, + 498, + 475, + 483, + 505, + 482, + 500, + 470, + 496, + 458, + 460, + 524, + 435, + 490, + 494, + 516, + 491, + 487, + 482, + 480, + 498, + 487, + 479, + 503, + 470, + 477, + 468, + 500, + 489, + 447, + 495, + 490, + 501, + 494, + 516, + 475, + 461, + 510, + 489, + 467, + 499, + 485, + 483, + 520, + 486, + 488, + 515, + 494, + 516, + 517, + 509, + 503, + 475, + 511, + 449, + 510, + 497, + 506, + 474, + 506, + 516, + 534, + 487, + 485, + 455, + 487, + 499, + 506, + 481, + 496, + 505, + 497, + 504, + 502, + 488, + 496, + 483, + 521, + 528, + 479, + 471, + 498, + 504, + 491, + 476, + 482, + 472, + 486, + 478, + 480, + 534, + 463, + 494, + 487, + 497, + 473, + 496, + 465, + 515, + 498, + 453, + 516, + 474, + 463, + 502, + 516, + 496, + 475, + 503, + 478, + 510, + 479, + 518, + 479, + 519, + 499, + 464, + 475, + 469, + 480, + 483, + 492, + 468, + 507, + 474, + 452, + 509, + 439, + 478, + 510, + 521, + 522, + 498, + 525, + 499, + 489, + 485, + 514, + 501, + 506, + 466, + 462, + 457, + 490, + 538, + 462, + 489, + 451, + 499, + 481, + 468, + 509, + 508, + 517, + 533, + 469, + 497, + 513, + 511, + 523, + 482, + 492, + 507, + 512, + 477, + 487, + 490, + 495, + 462, + 491, + 505, + 500, + 484, + 475, + 515, + 490, + 471, + 547, + 480, + 501, + 483, + 469, + 476, + 513, + 506, + 494, + 462, + 494, + 490, + 466, + 493, + 509, + 476, + 492, + 483, + 442, + 512, + 475, + 502, + 521, + 467, + 469, + 458, + 476, + 467, + 474, + 517, + 485, + 502, + 507, + 473, + 488, + 464, + 483, + 469, + 537, + 464, + 509, + 498, + 489, + 471, + 479, + 520, + 525, + 490, + 454, + 519, + 475, + 472, + 475, + 517, + 513, + 502, + 493, + 485, + 524, + 488, + 511, + 483, + 496, + 502, + 462, + 485, + 505, + 469, + 526, + 484, + 523, + 482, + 477, + 470, + 485, + 469, + 480, + 509, + 485, + 486, + 524, + 510, + 503, + 457, + 464, + 492, + 473, + 480, + 503, + 493, + 517, + 491, + 471, + 480, + 530, + 457, + 469, + 494, + 500, + 489, + 506, + 462, + 455, + 494, + 472, + 481, + 481, + 527, + 523, + 440, + 502, + 447, + 469, + 479, + 515, + 495, + 522, + 494, + 513, + 537, + 507, + 501, + 506, + 438, + 501, + 486, + 441, + 498, + 486, + 498, + 498, + 513, + 521, + 502, + 474, + 519, + 477, + 521, + 492, + 505, + 488, + 485, + 519, + 457, + 492, + 470, + 484, + 473, + 499, + 516, + 479, + 506, + 476, + 455, + 472, + 476, + 471, + 490, + 482, + 517, + 463, + 466, + 485, + 502, + 486, + 472, + 469, + 502, + 537, + 452, + 528, + 485, + 497, + 472, + 530, + 485, + 458, + 477, + 547, + 519, + 485, + 466, + 463, + 534, + 512, + 491, + 483, + 502, + 512, + 478, + 482, + 496, + 476, + 483, + 487, + 484, + 473, + 526, + 485, + 440, + 498, + 507, + 499, + 493, + 480, + 471, + 494, + 496, + 477, + 490, + 465, + 451, + 508, + 469, + 427, + 500, + 484, + 493, + 493, + 478, + 468, + 508, + 539, + 508, + 481, + 517, + 485, + 469, + 475, + 494, + 508, + 473, + 481, + 502, + 481, + 470, + 493, + 485, + 471, + 505, + 495, + 487, + 461, + 520, + 500, + 469, + 488, + 489, + 495, + 480, + 479, + 461, + 452, + 487, + 501, + 490, + 474, + 488, + 490, + 517, + 506, + 496, + 510, + 542, + 482, + 474, + 490, + 492, + 475, + 496, + 479, + 485, + 503, + 511, + 474, + 504, + 482, + 518, + 517, + 483, + 479, + 493, + 445, + 477, + 497, + 491, + 505, + 488, + 458, + 511, + 517, + 498, + 490, + 467, + 528, + 466, + 481, + 487, + 484, + 503, + 459, + 474, + 503, + 476, + 494, + 453, + 482, + 512, + 490, + 467, + 509, + 475, + 469, + 463, + 494, + 447, + 478, + 507, + 520, + 497, + 502, + 564, + 456, + 466, + 513, + 507, + 498, + 526, + 464, + 496, + 495, + 486, + 462, + 469, + 506, + 494, + 500, + 506, + 457, + 515, + 497, + 471, + 491, + 483, + 476, + 520, + 479, + 473, + 493, + 481, + 447, + 502, + 541, + 444, + 467, + 459, + 497, + 502, + 490, + 465, + 502, + 463, + 480, + 473, + 457, + 470, + 487, + 529, + 494, + 467, + 495, + 488, + 479, + 484, + 533, + 485, + 458, + 490, + 502, + 473, + 493, + 473, + 480, + 494, + 514, + 503, + 470, + 454, + 496, + 500, + 506, + 461, + 498, + 489, + 484, + 458, + 474, + 472, + 488, + 481, + 517, + 511, + 485, + 518, + 499, + 485, + 478, + 498, + 502, + 483, + 468, + 463, + 455, + 499, + 491 + ] + }, + { + "counters": [ + 529, + 495, + 464, + 493, + 511, + 495, + 481, + 475, + 475, + 471, + 481, + 520, + 471, + 499, + 479, + 524, + 517, + 483, + 474, + 486, + 530, + 516, + 499, + 464, + 501, + 540, + 452, + 486, + 496, + 495, + 510, + 517, + 483, + 507, + 485, + 466, + 496, + 436, + 506, + 501, + 484, + 479, + 472, + 486, + 441, + 515, + 515, + 479, + 482, + 499, + 483, + 487, + 475, + 507, + 504, + 495, + 485, + 466, + 534, + 489, + 533, + 475, + 511, + 492, + 489, + 502, + 465, + 486, + 462, + 463, + 520, + 508, + 488, + 477, + 478, + 485, + 478, + 515, + 507, + 513, + 522, + 478, + 452, + 482, + 470, + 464, + 492, + 459, + 465, + 491, + 494, + 507, + 489, + 470, + 448, + 455, + 531, + 490, + 516, + 498, + 488, + 483, + 469, + 521, + 476, + 516, + 478, + 468, + 489, + 456, + 492, + 485, + 513, + 481, + 503, + 488, + 492, + 492, + 484, + 497, + 479, + 472, + 457, + 466, + 478, + 482, + 468, + 468, + 460, + 475, + 495, + 479, + 481, + 464, + 496, + 466, + 468, + 474, + 487, + 487, + 477, + 513, + 468, + 473, + 463, + 523, + 480, + 454, + 479, + 439, + 492, + 474, + 468, + 502, + 489, + 515, + 490, + 485, + 456, + 461, + 506, + 491, + 457, + 471, + 499, + 473, + 526, + 506, + 514, + 466, + 454, + 470, + 494, + 500, + 487, + 485, + 508, + 459, + 518, + 498, + 496, + 514, + 463, + 496, + 466, + 461, + 507, + 493, + 481, + 479, + 491, + 508, + 485, + 472, + 524, + 482, + 473, + 504, + 495, + 531, + 509, + 483, + 490, + 462, + 505, + 469, + 484, + 446, + 502, + 484, + 496, + 463, + 485, + 442, + 479, + 518, + 510, + 463, + 452, + 488, + 516, + 514, + 507, + 448, + 501, + 496, + 484, + 451, + 543, + 535, + 509, + 532, + 474, + 457, + 504, + 513, + 503, + 460, + 456, + 468, + 475, + 509, + 496, + 465, + 501, + 526, + 481, + 494, + 472, + 513, + 503, + 505, + 451, + 491, + 513, + 508, + 504, + 479, + 475, + 477, + 469, + 512, + 518, + 492, + 514, + 458, + 497, + 484, + 509, + 493, + 500, + 474, + 470, + 496, + 468, + 495, + 491, + 504, + 505, + 460, + 435, + 489, + 459, + 478, + 505, + 538, + 485, + 465, + 511, + 473, + 480, + 459, + 495, + 480, + 475, + 476, + 482, + 500, + 499, + 490, + 470, + 455, + 502, + 448, + 491, + 450, + 457, + 484, + 487, + 503, + 523, + 454, + 501, + 506, + 483, + 492, + 468, + 464, + 483, + 486, + 491, + 483, + 466, + 480, + 482, + 498, + 497, + 512, + 464, + 526, + 533, + 471, + 481, + 453, + 484, + 516, + 521, + 503, + 501, + 509, + 491, + 477, + 481, + 516, + 466, + 466, + 463, + 503, + 519, + 503, + 493, + 469, + 484, + 477, + 449, + 454, + 488, + 482, + 511, + 466, + 467, + 488, + 455, + 491, + 463, + 479, + 486, + 508, + 485, + 473, + 482, + 445, + 457, + 487, + 466, + 486, + 505, + 490, + 483, + 490, + 468, + 466, + 519, + 498, + 508, + 499, + 446, + 478, + 505, + 495, + 472, + 502, + 465, + 470, + 481, + 462, + 502, + 499, + 469, + 505, + 513, + 485, + 504, + 491, + 470, + 467, + 488, + 503, + 499, + 477, + 510, + 458, + 518, + 484, + 481, + 531, + 482, + 523, + 470, + 509, + 491, + 496, + 491, + 477, + 443, + 484, + 486, + 463, + 485, + 479, + 495, + 453, + 480, + 504, + 484, + 458, + 498, + 498, + 487, + 536, + 499, + 496, + 516, + 480, + 492, + 470, + 492, + 446, + 524, + 484, + 493, + 508, + 486, + 535, + 494, + 477, + 504, + 505, + 473, + 472, + 523, + 501, + 507, + 514, + 454, + 513, + 501, + 502, + 490, + 477, + 476, + 452, + 477, + 483, + 481, + 464, + 504, + 476, + 500, + 511, + 501, + 470, + 470, + 496, + 502, + 475, + 502, + 504, + 480, + 497, + 475, + 486, + 491, + 508, + 523, + 519, + 479, + 473, + 452, + 450, + 479, + 487, + 512, + 490, + 503, + 482, + 492, + 486, + 464, + 500, + 488, + 538, + 517, + 487, + 497, + 442, + 484, + 487, + 523, + 507, + 460, + 459, + 503, + 508, + 459, + 515, + 515, + 506, + 502, + 468, + 489, + 482, + 461, + 524, + 473, + 465, + 500, + 513, + 474, + 496, + 476, + 490, + 505, + 517, + 449, + 495, + 494, + 523, + 492, + 484, + 460, + 505, + 499, + 494, + 498, + 460, + 496, + 496, + 488, + 463, + 523, + 490, + 509, + 522, + 499, + 477, + 486, + 460, + 468, + 463, + 494, + 509, + 482, + 489, + 477, + 497, + 498, + 518, + 450, + 444, + 486, + 501, + 503, + 473, + 490, + 516, + 506, + 458, + 504, + 464, + 452, + 476, + 475, + 474, + 463, + 500, + 454, + 529, + 508, + 491, + 481, + 475, + 471, + 485, + 479, + 468, + 492, + 450, + 481, + 487, + 468, + 481, + 507, + 463, + 448, + 492, + 522, + 498, + 456, + 499, + 468, + 455, + 491, + 504, + 502, + 519, + 502, + 533, + 482, + 459, + 469, + 502, + 502, + 463, + 468, + 530, + 453, + 472, + 483, + 503, + 475, + 478, + 484, + 506, + 493, + 485, + 475, + 510, + 465, + 483, + 472, + 493, + 496, + 506, + 457, + 515, + 485, + 456, + 465, + 499, + 497, + 472, + 477, + 491, + 472, + 448, + 470, + 474, + 513, + 485, + 465, + 488, + 447, + 490, + 478, + 532, + 498, + 547, + 454, + 490, + 473, + 504, + 492, + 471, + 497, + 486, + 512, + 484, + 444, + 495, + 479, + 460, + 482, + 516, + 505, + 480, + 500, + 511, + 472, + 495, + 487, + 483, + 486, + 504, + 463, + 441, + 542, + 475, + 496, + 485, + 489, + 463, + 475, + 468, + 487, + 507, + 501, + 479, + 443, + 492, + 453, + 460, + 497, + 474, + 468, + 452, + 501, + 503, + 482, + 456, + 472, + 469, + 456, + 470, + 504, + 497, + 513, + 454, + 520, + 494, + 506, + 493, + 501, + 459, + 504, + 531, + 466, + 514, + 465, + 519, + 475, + 500, + 504, + 504, + 476, + 500, + 470, + 477, + 496, + 484, + 484, + 506, + 481, + 515, + 491, + 508, + 506, + 477, + 497, + 478, + 515, + 500, + 516, + 451, + 480, + 487, + 514, + 478, + 457, + 489, + 447, + 496, + 465, + 516, + 469, + 465, + 496, + 500, + 492, + 478, + 461, + 496, + 479, + 498, + 511, + 476, + 474, + 482, + 471, + 486, + 497, + 517, + 506, + 451, + 510, + 494, + 484, + 529, + 535, + 474, + 500, + 510, + 501, + 499, + 526, + 471, + 496, + 484, + 491, + 471, + 488, + 495, + 443, + 474, + 487, + 476, + 493, + 435, + 494, + 494, + 475, + 477, + 500, + 502, + 479, + 474, + 505, + 466, + 472, + 466, + 463, + 541, + 522, + 481, + 501, + 481, + 489, + 454, + 520, + 477, + 467, + 505, + 466, + 503, + 480, + 538, + 490, + 481, + 509, + 479, + 493, + 512, + 473, + 481, + 467, + 468, + 476, + 489, + 489, + 488, + 483, + 475, + 512, + 488, + 512, + 479, + 459, + 475, + 457, + 538, + 483, + 455, + 498, + 471, + 486, + 468, + 504, + 450, + 514, + 485, + 460, + 510, + 532, + 467, + 467, + 496, + 478, + 475, + 443, + 456, + 488, + 509, + 463, + 508, + 490, + 488, + 469, + 514, + 495, + 470, + 512, + 428, + 495, + 483, + 484, + 509, + 532, + 499, + 491, + 515, + 472, + 504, + 508, + 478, + 523, + 539, + 481, + 478, + 515, + 505, + 460, + 500, + 470, + 507, + 483, + 438, + 479, + 510, + 475, + 481, + 522, + 471, + 508, + 560, + 496, + 457, + 453, + 483, + 484, + 466, + 493, + 519, + 457, + 478, + 486, + 489, + 503, + 494, + 484, + 483, + 497, + 532, + 475, + 499, + 474, + 481, + 487, + 432, + 495, + 480, + 503, + 505, + 515, + 466, + 479, + 463, + 498, + 502, + 451, + 503, + 482, + 484, + 499, + 503, + 462, + 483, + 510, + 537, + 480, + 468, + 508, + 490, + 514, + 451, + 507, + 517, + 517, + 478, + 461, + 436, + 502, + 511, + 503, + 475, + 513, + 485, + 462, + 489, + 476, + 500, + 496, + 497, + 470, + 501, + 492, + 455, + 465, + 528, + 468, + 519, + 463, + 516, + 501, + 476, + 491, + 444, + 480, + 499, + 491, + 472, + 510, + 484, + 497, + 494, + 479, + 519, + 505, + 527, + 506, + 488, + 494, + 499, + 514, + 468, + 476, + 477, + 513, + 496, + 471, + 532, + 486, + 469, + 488, + 527, + 489, + 473, + 475, + 504, + 485, + 520, + 479, + 472, + 508, + 475, + 493, + 498, + 464, + 446, + 471, + 463, + 471, + 500, + 491, + 490, + 543, + 500, + 513, + 521, + 473, + 481, + 467, + 425, + 461, + 476, + 455, + 472, + 510, + 489, + 469, + 510, + 499, + 489, + 469, + 495, + 488, + 514, + 467, + 525, + 497, + 512, + 517, + 500, + 479, + 461, + 479, + 517, + 522, + 491, + 478, + 508, + 498, + 458, + 504, + 480, + 460, + 481, + 527, + 528, + 484, + 474, + 481, + 462, + 462, + 483, + 497, + 486, + 494, + 535, + 466, + 441, + 495, + 489, + 469, + 497, + 510, + 519, + 475, + 483, + 503, + 473, + 509, + 511, + 487, + 464, + 495, + 468, + 501, + 498, + 509, + 502, + 480, + 488, + 511, + 451, + 483, + 518, + 484, + 530, + 488, + 543, + 487, + 476, + 474, + 479, + 519, + 514, + 500, + 493, + 506, + 469, + 457, + 514, + 469, + 475, + 476, + 490, + 505, + 472, + 511, + 499, + 469, + 482, + 456, + 510, + 475, + 483, + 500, + 453, + 504, + 483, + 498, + 457, + 481, + 507, + 493, + 459, + 480, + 463, + 479, + 475, + 454, + 517, + 484, + 473, + 464, + 460, + 505, + 515, + 483, + 520, + 504, + 466, + 497, + 491, + 490, + 500, + 500, + 487, + 493, + 467, + 501, + 463, + 501, + 450, + 457, + 503, + 486, + 468, + 483, + 482, + 494, + 497, + 468, + 483, + 447, + 454, + 457, + 474, + 461, + 473, + 500, + 463, + 496, + 510, + 475, + 506, + 475, + 468, + 488, + 504, + 475, + 479, + 462, + 505, + 509, + 530, + 527, + 477, + 492, + 445, + 500, + 469, + 465, + 526, + 505, + 486, + 496, + 497, + 462, + 516, + 509, + 479, + 523, + 473, + 502, + 456, + 496, + 488, + 459, + 453, + 472, + 466, + 439, + 526, + 484, + 482, + 533, + 496, + 480, + 494, + 459, + 503, + 477, + 475, + 504, + 499, + 458, + 496, + 502, + 494, + 465, + 479, + 522, + 448, + 472, + 509, + 465, + 524, + 497, + 490, + 482, + 539, + 467, + 483, + 510, + 498, + 486, + 504, + 522, + 481, + 487, + 448, + 444, + 514, + 498, + 459, + 484, + 511, + 519, + 475, + 501, + 492, + 518, + 476, + 493, + 501, + 498, + 480, + 501, + 489, + 483, + 498, + 527, + 464, + 535, + 551, + 486, + 460, + 501, + 501, + 478, + 503, + 479, + 511, + 418, + 504, + 508, + 474, + 476, + 443, + 464, + 490, + 498, + 476, + 466, + 498, + 449, + 504, + 494, + 484, + 455, + 470, + 496, + 519, + 490, + 510, + 516, + 473, + 487, + 504, + 496, + 475, + 486, + 499, + 458, + 505, + 526, + 470, + 494, + 538, + 505, + 468, + 497, + 488, + 515, + 508, + 493, + 471, + 467, + 484, + 468, + 516, + 518, + 469, + 497, + 521, + 472, + 523, + 495, + 494, + 487, + 502, + 496, + 489, + 493, + 453, + 510, + 495, + 468, + 492, + 477, + 509, + 511, + 449, + 508, + 490, + 476, + 473, + 464, + 495, + 516, + 475, + 550, + 470, + 445, + 481, + 504, + 468, + 513, + 474, + 485, + 495, + 499, + 504, + 505, + 472, + 474, + 510, + 482, + 453, + 547, + 484, + 473, + 505, + 496, + 495, + 517, + 504, + 480, + 506, + 543, + 476, + 515, + 470, + 493, + 472, + 465, + 527, + 510, + 508, + 453, + 466, + 492, + 463, + 503, + 494, + 447, + 468, + 538, + 479, + 464, + 500, + 505, + 474, + 496, + 479, + 470, + 507, + 480, + 492, + 494, + 482, + 471, + 492, + 456, + 502, + 480, + 493, + 453, + 475, + 466, + 510, + 512, + 480, + 469, + 524, + 508, + 487, + 472, + 476, + 479, + 531, + 462, + 509, + 434, + 478, + 489, + 489, + 527, + 441, + 536, + 497, + 508, + 494, + 502, + 459, + 443, + 513, + 503, + 470, + 511, + 453, + 495, + 517, + 543, + 455, + 509, + 431, + 523, + 476, + 538, + 488, + 481, + 487, + 505, + 527, + 458, + 494, + 511, + 424, + 511, + 506, + 482, + 467, + 481, + 538, + 457, + 487, + 520, + 479, + 508, + 491, + 472, + 499, + 518, + 460, + 486, + 494, + 505, + 505, + 489, + 520, + 473, + 487, + 488, + 448, + 499, + 498, + 493, + 491, + 450, + 507, + 521, + 500, + 442, + 476, + 497, + 483, + 467, + 490, + 538, + 479, + 448, + 509, + 469, + 533, + 473, + 447, + 501, + 503, + 484, + 475, + 504, + 455, + 519, + 466, + 497, + 470, + 497, + 490, + 501, + 510, + 479, + 504, + 487, + 498, + 490, + 474, + 464, + 484, + 470, + 536, + 486, + 469, + 496, + 511, + 490, + 492, + 539, + 496, + 519, + 455, + 476, + 463, + 432, + 491, + 523, + 495, + 463, + 516, + 514, + 518, + 500, + 485, + 501, + 515, + 492, + 508, + 453, + 507, + 449, + 473, + 492, + 506, + 494, + 529, + 502, + 499, + 525, + 488, + 495, + 486, + 527, + 492, + 501, + 512, + 456, + 462, + 468, + 492, + 480, + 499, + 460, + 448, + 500, + 491, + 485, + 470, + 455, + 443, + 481, + 480, + 488, + 485, + 476, + 463, + 514, + 483, + 515, + 464, + 460, + 465, + 476, + 447, + 475, + 521, + 500, + 458, + 495, + 468, + 495, + 487, + 517, + 473, + 479, + 501, + 498, + 491, + 519, + 496, + 508, + 478, + 496, + 516, + 489, + 486, + 491, + 487, + 494, + 461, + 487, + 496, + 488, + 510, + 479, + 479, + 499, + 476, + 482, + 500, + 522, + 488, + 476, + 493, + 504, + 517, + 509, + 505, + 472, + 511, + 452, + 477, + 469, + 493, + 542, + 472, + 498, + 475, + 509, + 499, + 486, + 484, + 516, + 524, + 477, + 490, + 550, + 511, + 452, + 493, + 532, + 489, + 463, + 496, + 467, + 439, + 486, + 497, + 476, + 493, + 498, + 521, + 475, + 521, + 423, + 469, + 515, + 484, + 492, + 496, + 459, + 444, + 473, + 463, + 494, + 453, + 469, + 510, + 473, + 457, + 492, + 533, + 548, + 485, + 466, + 496, + 485, + 466, + 510, + 454, + 475, + 512, + 491, + 510, + 495, + 495, + 490, + 488, + 417, + 479, + 471, + 502, + 479, + 488, + 501, + 497, + 493, + 468, + 502, + 479, + 497, + 512, + 455, + 457, + 504, + 474, + 495, + 517, + 476, + 492, + 475, + 490, + 502, + 467, + 493, + 540, + 512, + 470, + 490, + 517, + 515, + 463, + 480, + 500, + 495, + 438, + 468, + 488, + 478, + 504, + 468, + 499, + 468, + 514, + 484, + 512, + 519, + 490, + 476, + 483, + 513, + 483, + 491, + 505, + 493, + 493, + 493, + 465, + 497, + 477, + 497, + 468, + 448, + 496, + 483, + 480, + 482, + 495, + 483, + 506, + 518, + 486, + 501, + 482, + 442, + 489, + 516, + 487, + 458, + 496, + 527, + 467, + 454, + 485, + 489, + 496, + 469, + 468, + 523, + 470, + 480, + 477, + 471, + 506, + 480, + 522, + 499, + 479, + 483, + 537, + 466, + 485, + 490, + 534, + 468, + 500, + 514, + 494, + 489, + 480, + 511, + 509, + 488, + 544, + 492, + 478, + 536, + 447, + 465, + 490, + 457, + 501, + 501, + 533, + 498, + 486, + 456, + 491, + 487, + 477, + 494, + 501, + 492, + 472, + 459, + 491, + 490, + 461, + 499, + 485, + 489, + 476, + 477, + 503, + 476, + 483, + 470, + 496, + 450, + 467, + 472, + 500, + 488, + 482, + 471, + 490, + 494, + 482, + 486, + 551, + 502, + 510, + 496, + 495, + 524, + 521, + 504, + 483, + 488, + 487, + 489, + 506, + 525, + 469, + 495, + 473, + 462, + 495, + 472, + 507, + 480, + 507, + 517, + 509, + 503, + 504, + 478, + 541, + 471, + 517, + 520, + 516, + 506, + 515, + 479, + 496, + 473, + 469, + 459, + 488, + 487, + 463, + 493, + 492, + 484, + 487, + 462, + 475, + 493, + 541, + 492, + 480, + 492, + 522, + 522, + 463, + 490, + 485, + 540, + 482, + 500, + 520, + 468, + 529, + 478, + 522, + 527, + 465, + 430, + 501, + 485, + 474, + 517, + 509, + 491, + 484, + 468, + 505 + ] + } + ], + "default_value": 0 + }, + "null_count": 0, + "tot_col_size": 0, + "last_update_version": 417600050861703168, + "correlation": 0 + }, + "idx_b": { + "histogram": { + "ndv": 1000000, + "buckets": [ + { + "count": 8192, + "lower_bound": "A4AAAAAAAAAB", + "upper_bound": "A4AAAAAAACAA", + "repeats": 1 + }, + { + "count": 16384, + "lower_bound": "A4AAAAAAACAB", + "upper_bound": "A4AAAAAAAEAA", + "repeats": 1 + }, + { + "count": 24576, + "lower_bound": "A4AAAAAAAEAB", + "upper_bound": "A4AAAAAAAGAA", + "repeats": 1 + }, + { + "count": 32768, + "lower_bound": "A4AAAAAAAGAB", + "upper_bound": "A4AAAAAAAIAA", + "repeats": 1 + }, + { + "count": 40960, + "lower_bound": "A4AAAAAAAIAB", + "upper_bound": "A4AAAAAAAKAA", + "repeats": 1 + }, + { + "count": 49152, + "lower_bound": "A4AAAAAAAKAB", + "upper_bound": "A4AAAAAAAMAA", + "repeats": 1 + }, + { + "count": 57344, + "lower_bound": "A4AAAAAAAMAB", + "upper_bound": "A4AAAAAAAOAA", + "repeats": 1 + }, + { + "count": 65536, + "lower_bound": "A4AAAAAAAOAB", + "upper_bound": "A4AAAAAAAQAA", + "repeats": 1 + }, + { + "count": 73728, + "lower_bound": "A4AAAAAAAQAB", + "upper_bound": "A4AAAAAAASAA", + "repeats": 1 + }, + { + "count": 81920, + "lower_bound": "A4AAAAAAASAB", + "upper_bound": "A4AAAAAAAUAA", + "repeats": 1 + }, + { + "count": 90112, + "lower_bound": "A4AAAAAAAUAB", + "upper_bound": "A4AAAAAAAWAA", + "repeats": 1 + }, + { + "count": 98304, + "lower_bound": "A4AAAAAAAWAB", + "upper_bound": "A4AAAAAAAYAA", + "repeats": 1 + }, + { + "count": 106496, + "lower_bound": "A4AAAAAAAYAB", + "upper_bound": "A4AAAAAAAaAA", + "repeats": 1 + }, + { + "count": 114688, + "lower_bound": "A4AAAAAAAaAB", + "upper_bound": "A4AAAAAAAcAA", + "repeats": 1 + }, + { + "count": 122880, + "lower_bound": "A4AAAAAAAcAB", + "upper_bound": "A4AAAAAAAeAA", + "repeats": 1 + }, + { + "count": 131072, + "lower_bound": "A4AAAAAAAeAB", + "upper_bound": "A4AAAAAAAgAA", + "repeats": 1 + }, + { + "count": 139264, + "lower_bound": "A4AAAAAAAgAB", + "upper_bound": "A4AAAAAAAiAA", + "repeats": 1 + }, + { + "count": 147456, + "lower_bound": "A4AAAAAAAiAB", + "upper_bound": "A4AAAAAAAkAA", + "repeats": 1 + }, + { + "count": 155648, + "lower_bound": "A4AAAAAAAkAB", + "upper_bound": "A4AAAAAAAmAA", + "repeats": 1 + }, + { + "count": 163840, + "lower_bound": "A4AAAAAAAmAB", + "upper_bound": "A4AAAAAAAoAA", + "repeats": 1 + }, + { + "count": 172032, + "lower_bound": "A4AAAAAAAoAB", + "upper_bound": "A4AAAAAAAqAA", + "repeats": 1 + }, + { + "count": 180224, + "lower_bound": "A4AAAAAAAqAB", + "upper_bound": "A4AAAAAAAsAA", + "repeats": 1 + }, + { + "count": 188416, + "lower_bound": "A4AAAAAAAsAB", + "upper_bound": "A4AAAAAAAuAA", + "repeats": 1 + }, + { + "count": 196608, + "lower_bound": "A4AAAAAAAuAB", + "upper_bound": "A4AAAAAAAwAA", + "repeats": 1 + }, + { + "count": 204800, + "lower_bound": "A4AAAAAAAwAB", + "upper_bound": "A4AAAAAAAyAA", + "repeats": 1 + }, + { + "count": 212992, + "lower_bound": "A4AAAAAAAyAB", + "upper_bound": "A4AAAAAAA0AA", + "repeats": 1 + }, + { + "count": 221184, + "lower_bound": "A4AAAAAAA0AB", + "upper_bound": "A4AAAAAAA2AA", + "repeats": 1 + }, + { + "count": 229376, + "lower_bound": "A4AAAAAAA2AB", + "upper_bound": "A4AAAAAAA4AA", + "repeats": 1 + }, + { + "count": 237568, + "lower_bound": "A4AAAAAAA4AB", + "upper_bound": "A4AAAAAAA6AA", + "repeats": 1 + }, + { + "count": 245760, + "lower_bound": "A4AAAAAAA6AB", + "upper_bound": "A4AAAAAAA8AA", + "repeats": 1 + }, + { + "count": 253952, + "lower_bound": "A4AAAAAAA8AB", + "upper_bound": "A4AAAAAAA+AA", + "repeats": 1 + }, + { + "count": 262144, + "lower_bound": "A4AAAAAAA+AB", + "upper_bound": "A4AAAAAABAAA", + "repeats": 1 + }, + { + "count": 270336, + "lower_bound": "A4AAAAAABAAB", + "upper_bound": "A4AAAAAABCAA", + "repeats": 1 + }, + { + "count": 278528, + "lower_bound": "A4AAAAAABCAB", + "upper_bound": "A4AAAAAABEAA", + "repeats": 1 + }, + { + "count": 286720, + "lower_bound": "A4AAAAAABEAB", + "upper_bound": "A4AAAAAABGAA", + "repeats": 1 + }, + { + "count": 294912, + "lower_bound": "A4AAAAAABGAB", + "upper_bound": "A4AAAAAABIAA", + "repeats": 1 + }, + { + "count": 303104, + "lower_bound": "A4AAAAAABIAB", + "upper_bound": "A4AAAAAABKAA", + "repeats": 1 + }, + { + "count": 311296, + "lower_bound": "A4AAAAAABKAB", + "upper_bound": "A4AAAAAABMAA", + "repeats": 1 + }, + { + "count": 319488, + "lower_bound": "A4AAAAAABMAB", + "upper_bound": "A4AAAAAABOAA", + "repeats": 1 + }, + { + "count": 327680, + "lower_bound": "A4AAAAAABOAB", + "upper_bound": "A4AAAAAABQAA", + "repeats": 1 + }, + { + "count": 335872, + "lower_bound": "A4AAAAAABQAB", + "upper_bound": "A4AAAAAABSAA", + "repeats": 1 + }, + { + "count": 344064, + "lower_bound": "A4AAAAAABSAB", + "upper_bound": "A4AAAAAABUAA", + "repeats": 1 + }, + { + "count": 352256, + "lower_bound": "A4AAAAAABUAB", + "upper_bound": "A4AAAAAABWAA", + "repeats": 1 + }, + { + "count": 360448, + "lower_bound": "A4AAAAAABWAB", + "upper_bound": "A4AAAAAABYAA", + "repeats": 1 + }, + { + "count": 368640, + "lower_bound": "A4AAAAAABYAB", + "upper_bound": "A4AAAAAABaAA", + "repeats": 1 + }, + { + "count": 376832, + "lower_bound": "A4AAAAAABaAB", + "upper_bound": "A4AAAAAABcAA", + "repeats": 1 + }, + { + "count": 385024, + "lower_bound": "A4AAAAAABcAB", + "upper_bound": "A4AAAAAABeAA", + "repeats": 1 + }, + { + "count": 393216, + "lower_bound": "A4AAAAAABeAB", + "upper_bound": "A4AAAAAABgAA", + "repeats": 1 + }, + { + "count": 401408, + "lower_bound": "A4AAAAAABgAB", + "upper_bound": "A4AAAAAABiAA", + "repeats": 1 + }, + { + "count": 409600, + "lower_bound": "A4AAAAAABiAB", + "upper_bound": "A4AAAAAABkAA", + "repeats": 1 + }, + { + "count": 417792, + "lower_bound": "A4AAAAAABkAB", + "upper_bound": "A4AAAAAABmAA", + "repeats": 1 + }, + { + "count": 425984, + "lower_bound": "A4AAAAAABmAB", + "upper_bound": "A4AAAAAABoAA", + "repeats": 1 + }, + { + "count": 434176, + "lower_bound": "A4AAAAAABoAB", + "upper_bound": "A4AAAAAABqAA", + "repeats": 1 + }, + { + "count": 442368, + "lower_bound": "A4AAAAAABqAB", + "upper_bound": "A4AAAAAABsAA", + "repeats": 1 + }, + { + "count": 450560, + "lower_bound": "A4AAAAAABsAB", + "upper_bound": "A4AAAAAABuAA", + "repeats": 1 + }, + { + "count": 458752, + "lower_bound": "A4AAAAAABuAB", + "upper_bound": "A4AAAAAABwAA", + "repeats": 1 + }, + { + "count": 466944, + "lower_bound": "A4AAAAAABwAB", + "upper_bound": "A4AAAAAAByAA", + "repeats": 1 + }, + { + "count": 475136, + "lower_bound": "A4AAAAAAByAB", + "upper_bound": "A4AAAAAAB0AA", + "repeats": 1 + }, + { + "count": 483328, + "lower_bound": "A4AAAAAAB0AB", + "upper_bound": "A4AAAAAAB2AA", + "repeats": 1 + }, + { + "count": 491520, + "lower_bound": "A4AAAAAAB2AB", + "upper_bound": "A4AAAAAAB4AA", + "repeats": 1 + }, + { + "count": 499712, + "lower_bound": "A4AAAAAAB4AB", + "upper_bound": "A4AAAAAAB6AA", + "repeats": 1 + }, + { + "count": 507904, + "lower_bound": "A4AAAAAAB6AB", + "upper_bound": "A4AAAAAAB8AA", + "repeats": 1 + }, + { + "count": 516096, + "lower_bound": "A4AAAAAAB8AB", + "upper_bound": "A4AAAAAAB+AA", + "repeats": 1 + }, + { + "count": 524288, + "lower_bound": "A4AAAAAAB+AB", + "upper_bound": "A4AAAAAACAAA", + "repeats": 1 + }, + { + "count": 532480, + "lower_bound": "A4AAAAAACAAB", + "upper_bound": "A4AAAAAACCAA", + "repeats": 1 + }, + { + "count": 540672, + "lower_bound": "A4AAAAAACCAB", + "upper_bound": "A4AAAAAACEAA", + "repeats": 1 + }, + { + "count": 548864, + "lower_bound": "A4AAAAAACEAB", + "upper_bound": "A4AAAAAACGAA", + "repeats": 1 + }, + { + "count": 557056, + "lower_bound": "A4AAAAAACGAB", + "upper_bound": "A4AAAAAACIAA", + "repeats": 1 + }, + { + "count": 565248, + "lower_bound": "A4AAAAAACIAB", + "upper_bound": "A4AAAAAACKAA", + "repeats": 1 + }, + { + "count": 573440, + "lower_bound": "A4AAAAAACKAB", + "upper_bound": "A4AAAAAACMAA", + "repeats": 1 + }, + { + "count": 581632, + "lower_bound": "A4AAAAAACMAB", + "upper_bound": "A4AAAAAACOAA", + "repeats": 1 + }, + { + "count": 589824, + "lower_bound": "A4AAAAAACOAB", + "upper_bound": "A4AAAAAACQAA", + "repeats": 1 + }, + { + "count": 598016, + "lower_bound": "A4AAAAAACQAB", + "upper_bound": "A4AAAAAACSAA", + "repeats": 1 + }, + { + "count": 606208, + "lower_bound": "A4AAAAAACSAB", + "upper_bound": "A4AAAAAACUAA", + "repeats": 1 + }, + { + "count": 614400, + "lower_bound": "A4AAAAAACUAB", + "upper_bound": "A4AAAAAACWAA", + "repeats": 1 + }, + { + "count": 622592, + "lower_bound": "A4AAAAAACWAB", + "upper_bound": "A4AAAAAACYAA", + "repeats": 1 + }, + { + "count": 630784, + "lower_bound": "A4AAAAAACYAB", + "upper_bound": "A4AAAAAACaAA", + "repeats": 1 + }, + { + "count": 638976, + "lower_bound": "A4AAAAAACaAB", + "upper_bound": "A4AAAAAACcAA", + "repeats": 1 + }, + { + "count": 647168, + "lower_bound": "A4AAAAAACcAB", + "upper_bound": "A4AAAAAACeAA", + "repeats": 1 + }, + { + "count": 655360, + "lower_bound": "A4AAAAAACeAB", + "upper_bound": "A4AAAAAACgAA", + "repeats": 1 + }, + { + "count": 663552, + "lower_bound": "A4AAAAAACgAB", + "upper_bound": "A4AAAAAACiAA", + "repeats": 1 + }, + { + "count": 671744, + "lower_bound": "A4AAAAAACiAB", + "upper_bound": "A4AAAAAACkAA", + "repeats": 1 + }, + { + "count": 679936, + "lower_bound": "A4AAAAAACkAB", + "upper_bound": "A4AAAAAACmAA", + "repeats": 1 + }, + { + "count": 688128, + "lower_bound": "A4AAAAAACmAB", + "upper_bound": "A4AAAAAACoAA", + "repeats": 1 + }, + { + "count": 696320, + "lower_bound": "A4AAAAAACoAB", + "upper_bound": "A4AAAAAACqAA", + "repeats": 1 + }, + { + "count": 704512, + "lower_bound": "A4AAAAAACqAB", + "upper_bound": "A4AAAAAACsAA", + "repeats": 1 + }, + { + "count": 712704, + "lower_bound": "A4AAAAAACsAB", + "upper_bound": "A4AAAAAACuAA", + "repeats": 1 + }, + { + "count": 720896, + "lower_bound": "A4AAAAAACuAB", + "upper_bound": "A4AAAAAACwAA", + "repeats": 1 + }, + { + "count": 729088, + "lower_bound": "A4AAAAAACwAB", + "upper_bound": "A4AAAAAACyAA", + "repeats": 1 + }, + { + "count": 737280, + "lower_bound": "A4AAAAAACyAB", + "upper_bound": "A4AAAAAAC0AA", + "repeats": 1 + }, + { + "count": 745472, + "lower_bound": "A4AAAAAAC0AB", + "upper_bound": "A4AAAAAAC2AA", + "repeats": 1 + }, + { + "count": 753664, + "lower_bound": "A4AAAAAAC2AB", + "upper_bound": "A4AAAAAAC4AA", + "repeats": 1 + }, + { + "count": 761856, + "lower_bound": "A4AAAAAAC4AB", + "upper_bound": "A4AAAAAAC6AA", + "repeats": 1 + }, + { + "count": 766022, + "lower_bound": "A4AAAAAAC6AB", + "upper_bound": "A4AAAAAAC7BG", + "repeats": 1 + }, + { + "count": 768071, + "lower_bound": "A4AAAAAAC7BH", + "upper_bound": "A4AAAAAAC7hH", + "repeats": 1 + }, + { + "count": 772167, + "lower_bound": "A4AAAAAAC7hI", + "upper_bound": "A4AAAAAAC8hH", + "repeats": 1 + }, + { + "count": 776263, + "lower_bound": "A4AAAAAAC8hI", + "upper_bound": "A4AAAAAAC9hH", + "repeats": 1 + }, + { + "count": 780359, + "lower_bound": "A4AAAAAAC9hI", + "upper_bound": "A4AAAAAAC+hH", + "repeats": 1 + }, + { + "count": 784455, + "lower_bound": "A4AAAAAAC+hI", + "upper_bound": "A4AAAAAAC/hH", + "repeats": 1 + }, + { + "count": 788551, + "lower_bound": "A4AAAAAAC/hI", + "upper_bound": "A4AAAAAADAhH", + "repeats": 1 + }, + { + "count": 792647, + "lower_bound": "A4AAAAAADAhI", + "upper_bound": "A4AAAAAADBhH", + "repeats": 1 + }, + { + "count": 796743, + "lower_bound": "A4AAAAAADBhI", + "upper_bound": "A4AAAAAADChH", + "repeats": 1 + }, + { + "count": 800839, + "lower_bound": "A4AAAAAADChI", + "upper_bound": "A4AAAAAADDhH", + "repeats": 1 + }, + { + "count": 804935, + "lower_bound": "A4AAAAAADDhI", + "upper_bound": "A4AAAAAADEhH", + "repeats": 1 + }, + { + "count": 809031, + "lower_bound": "A4AAAAAADEhI", + "upper_bound": "A4AAAAAADFhH", + "repeats": 1 + }, + { + "count": 813127, + "lower_bound": "A4AAAAAADFhI", + "upper_bound": "A4AAAAAADGhH", + "repeats": 1 + }, + { + "count": 817223, + "lower_bound": "A4AAAAAADGhI", + "upper_bound": "A4AAAAAADHhH", + "repeats": 1 + }, + { + "count": 821319, + "lower_bound": "A4AAAAAADHhI", + "upper_bound": "A4AAAAAADIhH", + "repeats": 1 + }, + { + "count": 825415, + "lower_bound": "A4AAAAAADIhI", + "upper_bound": "A4AAAAAADJhH", + "repeats": 1 + }, + { + "count": 829511, + "lower_bound": "A4AAAAAADJhI", + "upper_bound": "A4AAAAAADKhH", + "repeats": 1 + }, + { + "count": 833607, + "lower_bound": "A4AAAAAADKhI", + "upper_bound": "A4AAAAAADLhH", + "repeats": 1 + }, + { + "count": 837703, + "lower_bound": "A4AAAAAADLhI", + "upper_bound": "A4AAAAAADMhH", + "repeats": 1 + }, + { + "count": 841799, + "lower_bound": "A4AAAAAADMhI", + "upper_bound": "A4AAAAAADNhH", + "repeats": 1 + }, + { + "count": 845895, + "lower_bound": "A4AAAAAADNhI", + "upper_bound": "A4AAAAAADOhH", + "repeats": 1 + }, + { + "count": 849991, + "lower_bound": "A4AAAAAADOhI", + "upper_bound": "A4AAAAAADPhH", + "repeats": 1 + }, + { + "count": 854087, + "lower_bound": "A4AAAAAADPhI", + "upper_bound": "A4AAAAAADQhH", + "repeats": 1 + }, + { + "count": 858183, + "lower_bound": "A4AAAAAADQhI", + "upper_bound": "A4AAAAAADRhH", + "repeats": 1 + }, + { + "count": 862279, + "lower_bound": "A4AAAAAADRhI", + "upper_bound": "A4AAAAAADShH", + "repeats": 1 + }, + { + "count": 866375, + "lower_bound": "A4AAAAAADShI", + "upper_bound": "A4AAAAAADThH", + "repeats": 1 + }, + { + "count": 870471, + "lower_bound": "A4AAAAAADThI", + "upper_bound": "A4AAAAAADUhH", + "repeats": 1 + }, + { + "count": 874567, + "lower_bound": "A4AAAAAADUhI", + "upper_bound": "A4AAAAAADVhH", + "repeats": 1 + }, + { + "count": 878663, + "lower_bound": "A4AAAAAADVhI", + "upper_bound": "A4AAAAAADWhH", + "repeats": 1 + }, + { + "count": 882759, + "lower_bound": "A4AAAAAADWhI", + "upper_bound": "A4AAAAAADXhH", + "repeats": 1 + }, + { + "count": 886855, + "lower_bound": "A4AAAAAADXhI", + "upper_bound": "A4AAAAAADYhH", + "repeats": 1 + }, + { + "count": 890951, + "lower_bound": "A4AAAAAADYhI", + "upper_bound": "A4AAAAAADZhH", + "repeats": 1 + }, + { + "count": 895047, + "lower_bound": "A4AAAAAADZhI", + "upper_bound": "A4AAAAAADahH", + "repeats": 1 + }, + { + "count": 899143, + "lower_bound": "A4AAAAAADahI", + "upper_bound": "A4AAAAAADbhH", + "repeats": 1 + }, + { + "count": 903239, + "lower_bound": "A4AAAAAADbhI", + "upper_bound": "A4AAAAAADchH", + "repeats": 1 + }, + { + "count": 907335, + "lower_bound": "A4AAAAAADchI", + "upper_bound": "A4AAAAAADdhH", + "repeats": 1 + }, + { + "count": 911431, + "lower_bound": "A4AAAAAADdhI", + "upper_bound": "A4AAAAAADehH", + "repeats": 1 + }, + { + "count": 915527, + "lower_bound": "A4AAAAAADehI", + "upper_bound": "A4AAAAAADfhH", + "repeats": 1 + }, + { + "count": 919623, + "lower_bound": "A4AAAAAADfhI", + "upper_bound": "A4AAAAAADghH", + "repeats": 1 + }, + { + "count": 923719, + "lower_bound": "A4AAAAAADghI", + "upper_bound": "A4AAAAAADhhH", + "repeats": 1 + }, + { + "count": 927815, + "lower_bound": "A4AAAAAADhhI", + "upper_bound": "A4AAAAAADihH", + "repeats": 1 + }, + { + "count": 931911, + "lower_bound": "A4AAAAAADihI", + "upper_bound": "A4AAAAAADjhH", + "repeats": 1 + }, + { + "count": 936007, + "lower_bound": "A4AAAAAADjhI", + "upper_bound": "A4AAAAAADkhH", + "repeats": 1 + }, + { + "count": 940103, + "lower_bound": "A4AAAAAADkhI", + "upper_bound": "A4AAAAAADlhH", + "repeats": 1 + }, + { + "count": 944199, + "lower_bound": "A4AAAAAADlhI", + "upper_bound": "A4AAAAAADmhH", + "repeats": 1 + }, + { + "count": 948295, + "lower_bound": "A4AAAAAADmhI", + "upper_bound": "A4AAAAAADnhH", + "repeats": 1 + }, + { + "count": 952391, + "lower_bound": "A4AAAAAADnhI", + "upper_bound": "A4AAAAAADohH", + "repeats": 1 + }, + { + "count": 956487, + "lower_bound": "A4AAAAAADohI", + "upper_bound": "A4AAAAAADphH", + "repeats": 1 + }, + { + "count": 960583, + "lower_bound": "A4AAAAAADphI", + "upper_bound": "A4AAAAAADqhH", + "repeats": 1 + }, + { + "count": 964679, + "lower_bound": "A4AAAAAADqhI", + "upper_bound": "A4AAAAAADrhH", + "repeats": 1 + }, + { + "count": 968775, + "lower_bound": "A4AAAAAADrhI", + "upper_bound": "A4AAAAAADshH", + "repeats": 1 + }, + { + "count": 972871, + "lower_bound": "A4AAAAAADshI", + "upper_bound": "A4AAAAAADthH", + "repeats": 1 + }, + { + "count": 976967, + "lower_bound": "A4AAAAAADthI", + "upper_bound": "A4AAAAAADuhH", + "repeats": 1 + }, + { + "count": 981063, + "lower_bound": "A4AAAAAADuhI", + "upper_bound": "A4AAAAAADvhH", + "repeats": 1 + }, + { + "count": 985159, + "lower_bound": "A4AAAAAADvhI", + "upper_bound": "A4AAAAAADwhH", + "repeats": 1 + }, + { + "count": 989255, + "lower_bound": "A4AAAAAADwhI", + "upper_bound": "A4AAAAAADxhH", + "repeats": 1 + }, + { + "count": 993351, + "lower_bound": "A4AAAAAADxhI", + "upper_bound": "A4AAAAAADyhH", + "repeats": 1 + }, + { + "count": 997447, + "lower_bound": "A4AAAAAADyhI", + "upper_bound": "A4AAAAAADzhH", + "repeats": 1 + }, + { + "count": 1000000, + "lower_bound": "A4AAAAAADzhI", + "upper_bound": "A4AAAAAAD0JA", + "repeats": 1 + } + ] + }, + "cm_sketch": { + "rows": [ + { + "counters": [ + 513, + 507, + 485, + 476, + 515, + 490, + 516, + 480, + 481, + 477, + 533, + 481, + 467, + 489, + 504, + 484, + 514, + 477, + 475, + 511, + 529, + 469, + 474, + 486, + 478, + 497, + 503, + 547, + 466, + 470, + 504, + 468, + 511, + 476, + 501, + 478, + 520, + 454, + 521, + 467, + 508, + 449, + 467, + 460, + 466, + 464, + 511, + 455, + 492, + 535, + 478, + 478, + 473, + 519, + 492, + 484, + 480, + 495, + 496, + 502, + 498, + 439, + 457, + 535, + 506, + 536, + 501, + 492, + 484, + 450, + 498, + 467, + 491, + 484, + 465, + 503, + 489, + 477, + 514, + 491, + 517, + 459, + 467, + 463, + 499, + 466, + 488, + 525, + 486, + 520, + 528, + 490, + 478, + 482, + 476, + 456, + 477, + 522, + 519, + 483, + 441, + 516, + 455, + 481, + 474, + 469, + 480, + 487, + 508, + 514, + 515, + 452, + 453, + 484, + 495, + 470, + 496, + 490, + 515, + 487, + 503, + 509, + 480, + 495, + 480, + 504, + 512, + 526, + 462, + 472, + 488, + 510, + 488, + 498, + 486, + 462, + 486, + 493, + 509, + 527, + 510, + 477, + 507, + 474, + 486, + 473, + 493, + 480, + 522, + 513, + 453, + 472, + 467, + 474, + 463, + 504, + 482, + 514, + 498, + 514, + 523, + 500, + 453, + 458, + 480, + 483, + 513, + 460, + 524, + 502, + 478, + 477, + 447, + 511, + 495, + 509, + 503, + 487, + 491, + 494, + 470, + 481, + 460, + 488, + 490, + 488, + 523, + 535, + 489, + 489, + 493, + 516, + 515, + 469, + 488, + 541, + 500, + 489, + 477, + 483, + 465, + 478, + 486, + 469, + 492, + 453, + 478, + 464, + 515, + 508, + 466, + 498, + 479, + 478, + 458, + 452, + 456, + 480, + 527, + 461, + 494, + 467, + 471, + 498, + 496, + 488, + 454, + 473, + 505, + 499, + 489, + 487, + 503, + 477, + 469, + 493, + 459, + 503, + 476, + 483, + 497, + 488, + 469, + 504, + 523, + 479, + 528, + 508, + 510, + 474, + 504, + 458, + 483, + 506, + 464, + 512, + 482, + 472, + 487, + 492, + 477, + 487, + 490, + 477, + 483, + 471, + 455, + 465, + 450, + 494, + 501, + 497, + 460, + 493, + 497, + 464, + 504, + 513, + 499, + 491, + 477, + 504, + 515, + 525, + 474, + 449, + 467, + 492, + 470, + 473, + 484, + 458, + 494, + 460, + 464, + 492, + 462, + 495, + 490, + 501, + 489, + 502, + 488, + 483, + 473, + 499, + 477, + 467, + 510, + 509, + 484, + 481, + 500, + 480, + 493, + 460, + 497, + 463, + 475, + 530, + 513, + 476, + 443, + 516, + 439, + 522, + 510, + 509, + 525, + 495, + 509, + 461, + 489, + 479, + 468, + 459, + 474, + 461, + 475, + 469, + 470, + 466, + 495, + 481, + 462, + 494, + 501, + 503, + 495, + 489, + 470, + 486, + 478, + 515, + 474, + 463, + 460, + 456, + 507, + 502, + 471, + 471, + 487, + 470, + 475, + 504, + 514, + 475, + 492, + 489, + 487, + 520, + 476, + 482, + 477, + 468, + 475, + 495, + 477, + 478, + 468, + 473, + 471, + 505, + 533, + 500, + 479, + 487, + 457, + 497, + 471, + 488, + 487, + 451, + 500, + 515, + 507, + 485, + 508, + 458, + 485, + 488, + 470, + 496, + 463, + 479, + 491, + 490, + 542, + 496, + 492, + 492, + 501, + 512, + 467, + 490, + 484, + 507, + 489, + 502, + 467, + 468, + 499, + 494, + 528, + 464, + 482, + 521, + 514, + 451, + 528, + 485, + 516, + 513, + 497, + 483, + 468, + 492, + 510, + 473, + 457, + 503, + 500, + 494, + 497, + 510, + 456, + 488, + 472, + 500, + 490, + 487, + 473, + 468, + 476, + 471, + 523, + 479, + 475, + 499, + 517, + 520, + 477, + 488, + 512, + 461, + 490, + 464, + 506, + 496, + 522, + 493, + 450, + 498, + 489, + 480, + 483, + 558, + 453, + 523, + 481, + 480, + 446, + 489, + 461, + 505, + 514, + 505, + 519, + 504, + 484, + 483, + 472, + 458, + 469, + 489, + 493, + 473, + 498, + 492, + 528, + 449, + 465, + 475, + 487, + 512, + 508, + 504, + 472, + 480, + 462, + 496, + 457, + 506, + 480, + 462, + 542, + 563, + 480, + 480, + 457, + 502, + 465, + 475, + 474, + 450, + 476, + 508, + 452, + 464, + 496, + 465, + 441, + 501, + 455, + 494, + 507, + 522, + 484, + 473, + 511, + 473, + 483, + 489, + 474, + 479, + 447, + 492, + 468, + 496, + 480, + 530, + 502, + 484, + 481, + 538, + 488, + 465, + 527, + 455, + 506, + 484, + 497, + 505, + 476, + 466, + 456, + 476, + 480, + 476, + 503, + 502, + 506, + 485, + 466, + 462, + 508, + 507, + 475, + 470, + 511, + 487, + 468, + 469, + 474, + 499, + 465, + 473, + 480, + 462, + 478, + 466, + 427, + 477, + 491, + 484, + 492, + 481, + 519, + 484, + 448, + 481, + 488, + 506, + 506, + 477, + 493, + 512, + 512, + 508, + 494, + 505, + 479, + 474, + 481, + 451, + 496, + 494, + 482, + 466, + 487, + 488, + 497, + 439, + 475, + 499, + 460, + 465, + 508, + 443, + 496, + 492, + 504, + 450, + 514, + 490, + 491, + 505, + 501, + 476, + 480, + 484, + 513, + 486, + 467, + 482, + 524, + 486, + 500, + 431, + 531, + 515, + 475, + 491, + 483, + 487, + 488, + 440, + 508, + 507, + 524, + 472, + 466, + 524, + 471, + 484, + 438, + 517, + 470, + 486, + 490, + 515, + 482, + 501, + 467, + 491, + 458, + 483, + 502, + 463, + 503, + 496, + 496, + 480, + 478, + 521, + 493, + 496, + 490, + 494, + 485, + 510, + 461, + 480, + 487, + 512, + 475, + 452, + 483, + 481, + 478, + 531, + 497, + 515, + 505, + 462, + 510, + 513, + 510, + 528, + 509, + 446, + 500, + 510, + 473, + 494, + 450, + 464, + 465, + 470, + 477, + 478, + 493, + 495, + 495, + 470, + 501, + 519, + 500, + 432, + 446, + 511, + 530, + 455, + 446, + 495, + 452, + 456, + 480, + 467, + 454, + 485, + 504, + 434, + 519, + 490, + 460, + 491, + 501, + 493, + 482, + 470, + 501, + 477, + 516, + 480, + 490, + 455, + 514, + 501, + 488, + 505, + 504, + 507, + 532, + 490, + 502, + 473, + 516, + 450, + 510, + 475, + 495, + 482, + 473, + 449, + 504, + 526, + 535, + 471, + 447, + 493, + 507, + 527, + 491, + 492, + 500, + 470, + 454, + 441, + 483, + 488, + 463, + 474, + 499, + 486, + 485, + 486, + 480, + 461, + 500, + 466, + 494, + 494, + 480, + 518, + 508, + 469, + 498, + 455, + 486, + 480, + 487, + 502, + 502, + 458, + 511, + 481, + 483, + 489, + 451, + 469, + 495, + 491, + 487, + 482, + 509, + 455, + 501, + 480, + 473, + 519, + 466, + 475, + 493, + 510, + 495, + 486, + 450, + 562, + 502, + 462, + 485, + 485, + 512, + 513, + 473, + 515, + 502, + 464, + 497, + 438, + 514, + 507, + 438, + 473, + 533, + 506, + 460, + 457, + 437, + 516, + 470, + 470, + 499, + 456, + 502, + 489, + 498, + 493, + 528, + 507, + 529, + 468, + 512, + 500, + 506, + 499, + 459, + 497, + 512, + 487, + 505, + 489, + 492, + 501, + 494, + 487, + 443, + 519, + 478, + 467, + 506, + 493, + 511, + 504, + 472, + 479, + 466, + 517, + 477, + 532, + 473, + 501, + 494, + 467, + 498, + 542, + 496, + 461, + 486, + 480, + 513, + 482, + 474, + 498, + 484, + 509, + 499, + 460, + 499, + 474, + 485, + 458, + 501, + 477, + 487, + 478, + 461, + 465, + 492, + 530, + 485, + 512, + 487, + 503, + 492, + 473, + 503, + 468, + 503, + 454, + 488, + 504, + 528, + 490, + 471, + 530, + 507, + 488, + 507, + 503, + 488, + 482, + 494, + 449, + 505, + 490, + 458, + 496, + 480, + 511, + 467, + 512, + 465, + 486, + 513, + 480, + 468, + 487, + 487, + 466, + 489, + 460, + 465, + 504, + 481, + 510, + 464, + 526, + 467, + 491, + 553, + 510, + 519, + 456, + 491, + 525, + 516, + 533, + 469, + 544, + 508, + 515, + 482, + 480, + 465, + 505, + 479, + 500, + 456, + 503, + 487, + 496, + 512, + 497, + 464, + 490, + 513, + 506, + 509, + 487, + 483, + 494, + 509, + 510, + 510, + 461, + 493, + 419, + 501, + 474, + 487, + 482, + 499, + 480, + 447, + 470, + 507, + 473, + 498, + 517, + 515, + 497, + 459, + 482, + 483, + 497, + 487, + 505, + 509, + 495, + 468, + 512, + 477, + 524, + 526, + 493, + 516, + 484, + 500, + 477, + 527, + 475, + 466, + 512, + 497, + 477, + 539, + 486, + 483, + 512, + 491, + 486, + 502, + 522, + 483, + 529, + 468, + 470, + 517, + 509, + 512, + 522, + 500, + 497, + 506, + 489, + 451, + 487, + 456, + 486, + 455, + 468, + 467, + 487, + 509, + 474, + 499, + 459, + 471, + 469, + 456, + 482, + 463, + 479, + 495, + 530, + 487, + 501, + 488, + 504, + 512, + 545, + 496, + 491, + 482, + 522, + 491, + 517, + 466, + 509, + 516, + 472, + 503, + 472, + 494, + 478, + 481, + 508, + 466, + 454, + 452, + 501, + 483, + 438, + 528, + 469, + 474, + 516, + 464, + 453, + 471, + 468, + 477, + 458, + 485, + 479, + 473, + 497, + 498, + 495, + 498, + 502, + 511, + 523, + 507, + 455, + 492, + 516, + 473, + 496, + 489, + 524, + 502, + 508, + 510, + 530, + 527, + 482, + 475, + 526, + 484, + 474, + 500, + 521, + 484, + 478, + 493, + 505, + 491, + 513, + 488, + 512, + 455, + 541, + 509, + 486, + 493, + 438, + 449, + 491, + 476, + 476, + 474, + 503, + 515, + 498, + 511, + 477, + 444, + 478, + 534, + 471, + 488, + 501, + 475, + 454, + 476, + 480, + 472, + 470, + 514, + 510, + 483, + 471, + 507, + 497, + 469, + 497, + 473, + 526, + 468, + 477, + 480, + 496, + 500, + 525, + 450, + 477, + 450, + 516, + 520, + 519, + 485, + 510, + 456, + 516, + 486, + 473, + 484, + 445, + 513, + 489, + 499, + 479, + 479, + 449, + 500, + 463, + 469, + 504, + 483, + 503, + 489, + 505, + 463, + 476, + 485, + 440, + 501, + 454, + 514, + 453, + 510, + 481, + 500, + 487, + 525, + 482, + 460, + 512, + 512, + 499, + 505, + 508, + 471, + 504, + 491, + 508, + 497, + 500, + 486, + 486, + 496, + 446, + 471, + 500, + 444, + 486, + 483, + 505, + 497, + 483, + 462, + 495, + 480, + 467, + 498, + 476, + 492, + 466, + 509, + 462, + 516, + 542, + 488, + 449, + 480, + 488, + 508, + 499, + 499, + 504, + 498, + 493, + 456, + 457, + 484, + 496, + 478, + 552, + 484, + 513, + 482, + 518, + 503, + 490, + 492, + 450, + 478, + 504, + 504, + 488, + 476, + 520, + 497, + 485, + 514, + 504, + 496, + 499, + 484, + 493, + 475, + 505, + 476, + 490, + 490, + 474, + 497, + 472, + 493, + 460, + 496, + 483, + 498, + 496, + 508, + 449, + 497, + 483, + 480, + 470, + 487, + 491, + 485, + 485, + 530, + 497, + 492, + 484, + 471, + 489, + 467, + 495, + 460, + 483, + 501, + 440, + 496, + 444, + 501, + 443, + 519, + 499, + 499, + 513, + 512, + 496, + 478, + 523, + 485, + 465, + 524, + 484, + 494, + 489, + 508, + 481, + 495, + 474, + 488, + 449, + 475, + 524, + 502, + 447, + 501, + 510, + 532, + 496, + 476, + 502, + 455, + 478, + 485, + 551, + 482, + 544, + 488, + 479, + 513, + 530, + 483, + 520, + 494, + 487, + 478, + 472, + 499, + 477, + 487, + 500, + 465, + 515, + 499, + 465, + 511, + 493, + 493, + 473, + 515, + 468, + 459, + 450, + 472, + 456, + 511, + 493, + 490, + 493, + 471, + 474, + 485, + 474, + 503, + 499, + 502, + 491, + 500, + 501, + 503, + 490, + 484, + 480, + 498, + 448, + 483, + 499, + 516, + 488, + 487, + 459, + 481, + 511, + 480, + 479, + 464, + 477, + 492, + 488, + 422, + 508, + 477, + 478, + 439, + 501, + 495, + 523, + 445, + 497, + 510, + 455, + 455, + 504, + 461, + 479, + 436, + 471, + 490, + 499, + 520, + 477, + 458, + 488, + 488, + 561, + 487, + 484, + 517, + 465, + 483, + 498, + 472, + 496, + 479, + 442, + 489, + 498, + 495, + 533, + 434, + 506, + 539, + 487, + 493, + 495, + 484, + 465, + 506, + 473, + 475, + 475, + 510, + 455, + 474, + 486, + 519, + 490, + 498, + 465, + 498, + 501, + 511, + 525, + 543, + 478, + 446, + 490, + 491, + 477, + 495, + 494, + 498, + 499, + 457, + 500, + 518, + 507, + 488, + 484, + 489, + 486, + 516, + 468, + 502, + 503, + 523, + 506, + 518, + 459, + 446, + 458, + 519, + 507, + 486, + 504, + 498, + 507, + 500, + 475, + 514, + 452, + 500, + 484, + 508, + 471, + 468, + 461, + 485, + 503, + 480, + 465, + 476, + 464, + 502, + 510, + 541, + 484, + 477, + 477, + 491, + 480, + 480, + 489, + 541, + 536, + 463, + 484, + 489, + 459, + 486, + 467, + 499, + 486, + 471, + 546, + 476, + 524, + 491, + 461, + 486, + 486, + 461, + 476, + 511, + 512, + 495, + 499, + 509, + 478, + 480, + 504, + 488, + 515, + 478, + 495, + 463, + 473, + 480, + 489, + 505, + 498, + 526, + 467, + 483, + 490, + 478, + 491, + 509, + 505, + 484, + 531, + 474, + 508, + 433, + 510, + 495, + 439, + 497, + 500, + 466, + 492, + 489, + 439, + 508, + 482, + 508, + 460, + 478, + 517, + 458, + 477, + 463, + 505, + 494, + 493, + 507, + 494, + 504, + 491, + 449, + 492, + 433, + 515, + 467, + 514, + 462, + 459, + 474, + 472, + 489, + 504, + 472, + 466, + 489, + 479, + 507, + 499, + 448, + 477, + 453, + 471, + 450, + 500, + 490, + 450, + 491, + 463, + 501, + 494, + 500, + 469, + 486, + 516, + 497, + 494, + 462, + 498, + 500, + 470, + 499, + 464, + 506, + 504, + 513, + 479, + 473, + 479, + 484, + 535, + 483, + 518, + 512, + 501, + 461, + 476, + 504, + 470, + 498, + 466, + 494, + 470, + 500, + 503, + 479, + 444, + 510, + 497, + 501, + 509, + 468, + 495, + 456, + 470, + 492, + 521, + 475, + 491, + 488, + 465, + 443, + 505, + 520, + 485, + 499, + 524, + 522, + 496, + 490, + 481, + 451, + 492, + 489, + 518, + 484, + 467, + 469, + 524, + 512, + 481, + 484, + 481, + 518, + 438, + 517, + 515, + 491, + 485, + 467, + 495, + 495, + 488, + 487, + 484, + 510, + 483, + 525, + 507, + 486, + 449, + 429, + 468, + 494, + 463, + 491, + 523, + 491, + 460, + 498, + 445, + 454, + 479, + 462, + 558, + 482, + 502, + 517, + 444, + 481, + 498, + 527, + 508, + 484, + 439, + 474, + 485, + 447, + 484, + 536, + 453, + 441, + 481, + 510, + 438, + 463, + 476, + 476, + 486, + 508, + 477, + 492, + 490, + 473, + 486, + 472, + 510, + 509, + 473, + 478, + 518, + 528, + 494, + 499, + 499, + 477, + 480, + 480, + 472, + 492, + 514, + 470, + 495, + 476, + 510, + 467, + 496, + 522, + 508, + 493, + 476, + 515, + 524, + 485, + 487, + 492, + 494, + 476, + 501, + 510, + 508, + 457, + 473, + 544, + 499, + 507, + 545, + 477, + 508, + 503, + 493, + 481, + 488, + 537, + 492, + 503, + 471, + 479, + 512, + 496, + 511, + 486, + 507, + 459, + 472, + 469, + 454, + 520, + 512, + 473, + 480, + 472, + 501, + 466, + 525, + 518, + 444, + 488, + 467, + 478, + 471, + 451, + 516, + 439, + 482, + 482, + 493, + 500, + 472, + 499, + 487, + 493, + 509, + 463, + 504, + 506, + 492, + 468, + 487, + 463, + 468, + 457, + 495, + 475, + 512, + 499, + 528, + 487, + 487, + 485, + 502, + 468, + 446, + 490, + 499, + 482, + 495, + 454, + 507, + 480, + 451, + 464, + 495, + 471, + 531, + 456, + 464, + 470, + 495, + 525, + 527, + 465, + 530, + 501, + 489, + 504, + 475, + 521, + 522, + 488, + 484, + 470, + 532, + 475, + 548, + 469, + 484, + 495, + 482, + 482, + 466, + 472, + 508, + 456, + 512, + 452, + 470, + 469, + 460, + 524, + 499, + 481, + 468, + 509, + 467, + 514, + 487, + 468, + 530, + 483, + 482, + 474, + 498, + 500, + 462, + 525, + 451, + 504, + 494, + 531, + 489, + 503, + 444, + 491, + 478, + 503, + 443, + 482, + 481, + 515, + 543, + 483, + 484, + 508, + 453, + 454, + 517, + 508, + 462, + 477, + 527, + 451, + 529, + 512, + 484, + 475, + 509, + 456, + 519, + 493, + 500, + 497, + 459, + 485, + 514, + 514, + 522 + ] + }, + { + "counters": [ + 472, + 498, + 470, + 465, + 475, + 525, + 470, + 465, + 469, + 495, + 486, + 460, + 487, + 480, + 495, + 479, + 448, + 477, + 477, + 486, + 461, + 456, + 504, + 474, + 482, + 473, + 492, + 509, + 465, + 523, + 451, + 513, + 502, + 462, + 461, + 488, + 523, + 469, + 513, + 501, + 484, + 521, + 463, + 487, + 510, + 446, + 528, + 529, + 467, + 514, + 493, + 513, + 476, + 490, + 461, + 476, + 483, + 488, + 445, + 482, + 516, + 529, + 471, + 454, + 499, + 496, + 483, + 467, + 504, + 481, + 495, + 488, + 489, + 474, + 498, + 510, + 486, + 497, + 510, + 479, + 491, + 472, + 492, + 500, + 505, + 502, + 519, + 461, + 490, + 514, + 498, + 492, + 524, + 492, + 473, + 474, + 520, + 470, + 444, + 509, + 483, + 510, + 508, + 476, + 484, + 484, + 475, + 468, + 475, + 469, + 466, + 471, + 439, + 493, + 479, + 499, + 468, + 487, + 460, + 513, + 480, + 531, + 500, + 527, + 460, + 490, + 475, + 497, + 508, + 484, + 486, + 487, + 488, + 492, + 520, + 477, + 504, + 470, + 509, + 485, + 496, + 466, + 476, + 479, + 501, + 494, + 495, + 485, + 510, + 499, + 514, + 485, + 494, + 474, + 495, + 465, + 480, + 475, + 479, + 548, + 466, + 517, + 507, + 507, + 458, + 460, + 507, + 471, + 493, + 472, + 497, + 509, + 504, + 498, + 488, + 484, + 470, + 453, + 495, + 485, + 467, + 525, + 485, + 522, + 469, + 500, + 441, + 467, + 509, + 469, + 513, + 452, + 489, + 498, + 497, + 435, + 476, + 500, + 510, + 520, + 475, + 461, + 501, + 513, + 488, + 480, + 494, + 481, + 493, + 501, + 470, + 504, + 475, + 500, + 499, + 471, + 469, + 519, + 484, + 478, + 434, + 448, + 522, + 445, + 489, + 515, + 463, + 511, + 483, + 484, + 468, + 480, + 488, + 482, + 476, + 469, + 462, + 470, + 477, + 500, + 475, + 503, + 480, + 539, + 473, + 498, + 522, + 475, + 502, + 526, + 474, + 464, + 493, + 449, + 527, + 464, + 510, + 511, + 466, + 489, + 493, + 512, + 478, + 498, + 469, + 515, + 529, + 506, + 488, + 506, + 535, + 460, + 509, + 489, + 473, + 478, + 503, + 461, + 484, + 517, + 426, + 469, + 473, + 470, + 477, + 454, + 505, + 495, + 476, + 462, + 495, + 486, + 468, + 488, + 498, + 463, + 468, + 503, + 530, + 541, + 488, + 475, + 497, + 502, + 464, + 516, + 492, + 505, + 476, + 454, + 514, + 492, + 463, + 506, + 488, + 487, + 493, + 497, + 498, + 511, + 517, + 490, + 499, + 474, + 458, + 485, + 490, + 504, + 489, + 487, + 468, + 503, + 483, + 504, + 484, + 499, + 489, + 537, + 507, + 509, + 527, + 521, + 471, + 501, + 459, + 539, + 470, + 491, + 474, + 511, + 479, + 474, + 525, + 510, + 478, + 474, + 476, + 481, + 519, + 462, + 492, + 471, + 504, + 472, + 478, + 483, + 448, + 504, + 519, + 479, + 442, + 472, + 492, + 477, + 545, + 472, + 498, + 420, + 500, + 469, + 495, + 508, + 485, + 502, + 492, + 470, + 468, + 472, + 481, + 470, + 493, + 466, + 496, + 482, + 500, + 513, + 522, + 484, + 496, + 476, + 487, + 503, + 491, + 507, + 461, + 503, + 496, + 478, + 515, + 470, + 505, + 542, + 487, + 502, + 503, + 482, + 454, + 509, + 510, + 460, + 514, + 521, + 465, + 491, + 444, + 471, + 470, + 488, + 495, + 492, + 492, + 514, + 521, + 479, + 492, + 538, + 454, + 482, + 486, + 472, + 505, + 523, + 514, + 516, + 488, + 523, + 505, + 480, + 477, + 503, + 487, + 513, + 463, + 509, + 480, + 515, + 497, + 460, + 532, + 474, + 445, + 450, + 501, + 530, + 507, + 495, + 499, + 492, + 495, + 477, + 487, + 478, + 506, + 476, + 510, + 493, + 440, + 490, + 450, + 522, + 484, + 497, + 494, + 487, + 477, + 482, + 460, + 445, + 474, + 481, + 510, + 504, + 478, + 503, + 484, + 497, + 467, + 537, + 469, + 523, + 521, + 472, + 489, + 464, + 476, + 482, + 527, + 491, + 483, + 476, + 500, + 496, + 505, + 462, + 472, + 453, + 493, + 506, + 458, + 446, + 499, + 477, + 473, + 532, + 495, + 466, + 498, + 484, + 491, + 485, + 486, + 478, + 460, + 513, + 482, + 474, + 470, + 482, + 463, + 490, + 479, + 513, + 496, + 499, + 458, + 458, + 490, + 483, + 508, + 475, + 485, + 486, + 491, + 538, + 493, + 486, + 516, + 496, + 498, + 490, + 483, + 509, + 476, + 497, + 463, + 489, + 488, + 496, + 498, + 475, + 510, + 487, + 457, + 451, + 515, + 554, + 488, + 467, + 461, + 463, + 486, + 488, + 482, + 500, + 512, + 509, + 468, + 467, + 474, + 523, + 488, + 480, + 471, + 479, + 502, + 487, + 465, + 472, + 499, + 478, + 478, + 487, + 479, + 479, + 520, + 520, + 491, + 472, + 511, + 499, + 492, + 469, + 491, + 500, + 497, + 465, + 530, + 492, + 450, + 490, + 451, + 505, + 538, + 491, + 493, + 513, + 468, + 452, + 491, + 487, + 514, + 496, + 461, + 471, + 504, + 488, + 504, + 511, + 502, + 439, + 526, + 483, + 491, + 506, + 503, + 505, + 498, + 498, + 477, + 487, + 456, + 447, + 522, + 517, + 492, + 478, + 485, + 522, + 526, + 491, + 447, + 465, + 491, + 457, + 451, + 490, + 455, + 436, + 435, + 468, + 480, + 491, + 503, + 512, + 472, + 524, + 458, + 502, + 443, + 476, + 483, + 495, + 490, + 490, + 463, + 508, + 476, + 466, + 508, + 479, + 516, + 518, + 513, + 455, + 490, + 532, + 490, + 487, + 453, + 485, + 527, + 480, + 508, + 500, + 459, + 471, + 455, + 519, + 488, + 487, + 492, + 468, + 485, + 488, + 460, + 486, + 502, + 520, + 508, + 483, + 473, + 497, + 486, + 488, + 452, + 434, + 493, + 526, + 483, + 467, + 476, + 488, + 469, + 501, + 500, + 464, + 480, + 526, + 475, + 526, + 422, + 519, + 468, + 503, + 528, + 499, + 500, + 470, + 523, + 504, + 493, + 474, + 482, + 485, + 480, + 527, + 498, + 472, + 516, + 439, + 481, + 474, + 505, + 491, + 482, + 453, + 477, + 514, + 539, + 491, + 506, + 533, + 510, + 490, + 514, + 472, + 518, + 492, + 467, + 504, + 520, + 460, + 496, + 532, + 511, + 500, + 473, + 476, + 448, + 500, + 461, + 467, + 500, + 549, + 488, + 507, + 477, + 494, + 464, + 467, + 511, + 515, + 461, + 483, + 485, + 485, + 493, + 509, + 510, + 426, + 486, + 521, + 503, + 509, + 452, + 495, + 506, + 524, + 480, + 452, + 510, + 485, + 506, + 482, + 519, + 481, + 508, + 492, + 503, + 475, + 494, + 487, + 485, + 483, + 502, + 524, + 500, + 476, + 522, + 500, + 514, + 454, + 489, + 494, + 509, + 510, + 496, + 521, + 477, + 483, + 499, + 424, + 454, + 477, + 485, + 510, + 496, + 519, + 521, + 506, + 503, + 485, + 423, + 495, + 507, + 465, + 490, + 490, + 494, + 515, + 501, + 508, + 498, + 460, + 455, + 495, + 510, + 510, + 504, + 505, + 475, + 467, + 488, + 479, + 421, + 503, + 506, + 455, + 454, + 499, + 475, + 510, + 500, + 490, + 507, + 510, + 480, + 483, + 504, + 503, + 450, + 495, + 508, + 477, + 493, + 474, + 519, + 466, + 489, + 479, + 513, + 481, + 444, + 469, + 504, + 498, + 491, + 463, + 450, + 506, + 516, + 482, + 494, + 480, + 530, + 492, + 512, + 479, + 491, + 468, + 486, + 474, + 471, + 471, + 502, + 524, + 493, + 508, + 487, + 492, + 489, + 508, + 471, + 500, + 503, + 495, + 461, + 482, + 462, + 496, + 473, + 459, + 498, + 508, + 509, + 504, + 488, + 473, + 486, + 492, + 484, + 509, + 485, + 482, + 464, + 532, + 485, + 492, + 466, + 464, + 491, + 520, + 496, + 480, + 473, + 494, + 449, + 478, + 503, + 479, + 482, + 483, + 479, + 506, + 462, + 491, + 475, + 483, + 488, + 475, + 468, + 508, + 501, + 464, + 501, + 492, + 492, + 461, + 513, + 479, + 534, + 477, + 476, + 470, + 517, + 466, + 470, + 485, + 454, + 477, + 424, + 468, + 507, + 485, + 425, + 514, + 467, + 500, + 474, + 491, + 501, + 478, + 482, + 483, + 501, + 463, + 492, + 466, + 488, + 460, + 451, + 465, + 476, + 479, + 513, + 468, + 471, + 472, + 488, + 505, + 502, + 469, + 472, + 469, + 520, + 514, + 453, + 498, + 487, + 514, + 526, + 451, + 473, + 527, + 516, + 482, + 542, + 491, + 464, + 470, + 533, + 464, + 473, + 468, + 532, + 509, + 459, + 520, + 488, + 446, + 520, + 492, + 459, + 494, + 491, + 519, + 487, + 491, + 496, + 430, + 499, + 507, + 492, + 514, + 506, + 486, + 499, + 463, + 493, + 486, + 447, + 508, + 504, + 523, + 493, + 481, + 468, + 503, + 481, + 454, + 483, + 498, + 518, + 488, + 462, + 477, + 518, + 476, + 466, + 485, + 489, + 514, + 461, + 503, + 470, + 458, + 464, + 488, + 501, + 508, + 484, + 474, + 479, + 488, + 518, + 510, + 520, + 495, + 505, + 500, + 503, + 450, + 447, + 512, + 486, + 489, + 486, + 530, + 473, + 508, + 472, + 507, + 472, + 483, + 519, + 485, + 490, + 464, + 499, + 489, + 447, + 481, + 479, + 455, + 470, + 474, + 463, + 487, + 457, + 469, + 502, + 495, + 512, + 482, + 501, + 483, + 469, + 523, + 482, + 521, + 473, + 480, + 521, + 504, + 498, + 513, + 492, + 506, + 499, + 509, + 503, + 464, + 495, + 470, + 481, + 457, + 458, + 496, + 493, + 490, + 491, + 505, + 496, + 467, + 470, + 481, + 470, + 507, + 532, + 520, + 462, + 482, + 488, + 425, + 466, + 493, + 492, + 503, + 482, + 474, + 465, + 461, + 468, + 495, + 513, + 533, + 486, + 495, + 491, + 466, + 494, + 487, + 463, + 489, + 503, + 494, + 514, + 503, + 482, + 455, + 501, + 536, + 521, + 493, + 474, + 490, + 502, + 482, + 492, + 476, + 482, + 479, + 521, + 514, + 443, + 526, + 484, + 498, + 457, + 479, + 502, + 437, + 502, + 497, + 501, + 482, + 457, + 450, + 481, + 457, + 517, + 518, + 512, + 491, + 494, + 490, + 470, + 482, + 500, + 479, + 491, + 512, + 500, + 526, + 505, + 483, + 495, + 480, + 535, + 482, + 511, + 511, + 463, + 472, + 503, + 511, + 505, + 496, + 448, + 508, + 492, + 481, + 461, + 466, + 484, + 475, + 471, + 464, + 475, + 482, + 463, + 483, + 545, + 469, + 486, + 491, + 528, + 517, + 491, + 504, + 451, + 476, + 467, + 487, + 503, + 480, + 474, + 506, + 491, + 473, + 518, + 425, + 482, + 467, + 521, + 471, + 494, + 454, + 472, + 535, + 476, + 494, + 516, + 496, + 489, + 495, + 448, + 520, + 482, + 520, + 498, + 472, + 504, + 502, + 463, + 485, + 477, + 520, + 506, + 470, + 466, + 497, + 511, + 482, + 457, + 491, + 516, + 495, + 512, + 447, + 472, + 474, + 507, + 516, + 513, + 515, + 462, + 485, + 480, + 471, + 520, + 467, + 527, + 467, + 475, + 509, + 484, + 464, + 487, + 507, + 504, + 519, + 484, + 445, + 490, + 499, + 477, + 497, + 548, + 466, + 476, + 505, + 487, + 501, + 509, + 488, + 489, + 510, + 474, + 500, + 516, + 492, + 501, + 477, + 525, + 530, + 503, + 498, + 467, + 480, + 476, + 517, + 523, + 497, + 469, + 457, + 466, + 472, + 469, + 481, + 487, + 451, + 488, + 482, + 504, + 481, + 489, + 490, + 492, + 499, + 505, + 517, + 496, + 517, + 477, + 427, + 473, + 506, + 475, + 486, + 507, + 472, + 467, + 520, + 474, + 452, + 476, + 494, + 478, + 481, + 485, + 477, + 441, + 508, + 519, + 477, + 490, + 460, + 466, + 517, + 470, + 468, + 525, + 497, + 519, + 475, + 492, + 466, + 482, + 523, + 476, + 502, + 483, + 466, + 475, + 487, + 479, + 498, + 472, + 463, + 479, + 488, + 460, + 477, + 478, + 451, + 465, + 484, + 503, + 521, + 522, + 476, + 503, + 477, + 476, + 477, + 513, + 506, + 465, + 481, + 462, + 485, + 509, + 498, + 510, + 483, + 511, + 510, + 500, + 478, + 488, + 487, + 526, + 506, + 525, + 489, + 527, + 484, + 442, + 506, + 481, + 506, + 502, + 498, + 435, + 486, + 495, + 487, + 496, + 487, + 457, + 513, + 540, + 469, + 496, + 494, + 488, + 499, + 471, + 471, + 485, + 505, + 453, + 486, + 491, + 507, + 475, + 512, + 482, + 462, + 486, + 476, + 466, + 445, + 490, + 519, + 484, + 492, + 496, + 489, + 509, + 520, + 468, + 510, + 505, + 473, + 486, + 538, + 495, + 484, + 477, + 481, + 491, + 535, + 474, + 494, + 496, + 497, + 475, + 472, + 485, + 510, + 481, + 564, + 487, + 466, + 502, + 496, + 498, + 514, + 483, + 495, + 514, + 497, + 482, + 492, + 479, + 487, + 482, + 471, + 506, + 488, + 515, + 464, + 511, + 511, + 502, + 504, + 451, + 505, + 470, + 498, + 488, + 498, + 502, + 444, + 464, + 480, + 500, + 503, + 521, + 473, + 482, + 469, + 472, + 538, + 470, + 492, + 491, + 482, + 488, + 497, + 499, + 465, + 462, + 496, + 469, + 492, + 481, + 501, + 456, + 498, + 439, + 533, + 490, + 449, + 473, + 491, + 476, + 501, + 492, + 489, + 461, + 448, + 481, + 486, + 456, + 484, + 492, + 474, + 515, + 471, + 458, + 523, + 483, + 513, + 484, + 472, + 486, + 518, + 443, + 463, + 507, + 487, + 451, + 490, + 485, + 488, + 495, + 525, + 484, + 500, + 478, + 504, + 470, + 491, + 449, + 471, + 477, + 472, + 468, + 506, + 507, + 520, + 473, + 520, + 465, + 507, + 480, + 475, + 491, + 487, + 495, + 473, + 440, + 495, + 473, + 433, + 505, + 506, + 516, + 466, + 462, + 505, + 472, + 482, + 494, + 512, + 497, + 476, + 515, + 466, + 491, + 466, + 511, + 477, + 496, + 535, + 513, + 458, + 509, + 477, + 522, + 485, + 471, + 440, + 504, + 480, + 479, + 485, + 474, + 485, + 490, + 518, + 507, + 508, + 511, + 454, + 475, + 502, + 480, + 476, + 490, + 507, + 485, + 484, + 494, + 530, + 486, + 484, + 523, + 458, + 476, + 473, + 452, + 472, + 461, + 474, + 500, + 518, + 487, + 508, + 505, + 474, + 484, + 532, + 462, + 462, + 472, + 445, + 489, + 508, + 517, + 474, + 484, + 483, + 456, + 482, + 488, + 452, + 511, + 449, + 501, + 489, + 454, + 477, + 537, + 484, + 485, + 516, + 463, + 507, + 500, + 479, + 480, + 483, + 491, + 473, + 443, + 483, + 482, + 497, + 486, + 502, + 497, + 530, + 515, + 552, + 477, + 517, + 478, + 513, + 490, + 536, + 501, + 486, + 503, + 499, + 502, + 514, + 521, + 493, + 514, + 471, + 459, + 503, + 466, + 465, + 516, + 532, + 496, + 495, + 513, + 469, + 476, + 476, + 501, + 501, + 491, + 479, + 469, + 463, + 468, + 505, + 520, + 509, + 474, + 489, + 483, + 504, + 494, + 497, + 493, + 506, + 516, + 503, + 472, + 456, + 518, + 467, + 458, + 487, + 458, + 493, + 525, + 480, + 453, + 485, + 484, + 491, + 493, + 478, + 482, + 529, + 484, + 486, + 471, + 465, + 481, + 498, + 511, + 530, + 486, + 464, + 461, + 470, + 489, + 516, + 507, + 495, + 493, + 492, + 445, + 483, + 517, + 518, + 460, + 481, + 478, + 498, + 480, + 494, + 505, + 516, + 460, + 448, + 477, + 460, + 490, + 490, + 465, + 496, + 535, + 476, + 507, + 468, + 480, + 508, + 505, + 534, + 537, + 464, + 502, + 481, + 497, + 481, + 477, + 487, + 467, + 483, + 491, + 512, + 479, + 489, + 507, + 484, + 478, + 451, + 472, + 470, + 509, + 486, + 464, + 478, + 481, + 520, + 492, + 480, + 459, + 489, + 437, + 495, + 479, + 489, + 486, + 474, + 458, + 470, + 479, + 464, + 481, + 468, + 516, + 479, + 509, + 501, + 477, + 523, + 481, + 487, + 507, + 517, + 487, + 518, + 508, + 518, + 484, + 496, + 487, + 507, + 479, + 460, + 491, + 460, + 523, + 520, + 471, + 505, + 490, + 451, + 468, + 494, + 482, + 491, + 462, + 490, + 434, + 485, + 496, + 466, + 482, + 490, + 480, + 452, + 487, + 471, + 468, + 481, + 538, + 475, + 482, + 495, + 491, + 489, + 493, + 469, + 483, + 480, + 508, + 516, + 473, + 487, + 513, + 493, + 506, + 489, + 478, + 480, + 469, + 455, + 487, + 509, + 473, + 490, + 512, + 510, + 488, + 517 + ] + }, + { + "counters": [ + 492, + 496, + 514, + 481, + 463, + 481, + 487, + 491, + 438, + 495, + 531, + 489, + 445, + 508, + 527, + 577, + 468, + 537, + 474, + 503, + 439, + 514, + 541, + 515, + 463, + 470, + 483, + 466, + 484, + 522, + 492, + 468, + 497, + 523, + 500, + 492, + 472, + 503, + 506, + 469, + 496, + 509, + 502, + 439, + 476, + 508, + 483, + 512, + 502, + 463, + 488, + 453, + 482, + 503, + 542, + 524, + 483, + 461, + 478, + 482, + 496, + 503, + 510, + 455, + 513, + 509, + 437, + 467, + 483, + 483, + 458, + 539, + 503, + 476, + 491, + 504, + 520, + 474, + 485, + 507, + 485, + 524, + 507, + 459, + 485, + 531, + 497, + 474, + 469, + 499, + 506, + 472, + 491, + 486, + 497, + 525, + 501, + 491, + 510, + 487, + 471, + 490, + 478, + 486, + 493, + 471, + 449, + 474, + 494, + 452, + 509, + 484, + 473, + 493, + 489, + 510, + 454, + 479, + 482, + 471, + 505, + 485, + 472, + 522, + 493, + 459, + 499, + 517, + 480, + 487, + 459, + 488, + 482, + 460, + 491, + 475, + 476, + 506, + 466, + 491, + 511, + 477, + 485, + 469, + 511, + 486, + 514, + 513, + 465, + 531, + 464, + 483, + 488, + 468, + 484, + 499, + 487, + 502, + 475, + 492, + 495, + 510, + 493, + 486, + 513, + 488, + 419, + 486, + 520, + 528, + 493, + 535, + 443, + 463, + 429, + 548, + 467, + 452, + 535, + 485, + 457, + 436, + 473, + 510, + 484, + 479, + 505, + 548, + 486, + 526, + 518, + 493, + 532, + 483, + 471, + 491, + 491, + 507, + 458, + 482, + 515, + 489, + 450, + 502, + 457, + 467, + 474, + 488, + 524, + 469, + 474, + 510, + 489, + 470, + 493, + 494, + 451, + 509, + 522, + 466, + 484, + 467, + 524, + 483, + 502, + 513, + 464, + 536, + 477, + 489, + 479, + 457, + 516, + 482, + 524, + 460, + 439, + 509, + 508, + 511, + 487, + 500, + 539, + 505, + 485, + 477, + 535, + 464, + 495, + 497, + 526, + 468, + 524, + 488, + 459, + 482, + 507, + 476, + 498, + 454, + 506, + 457, + 485, + 510, + 488, + 511, + 502, + 513, + 503, + 511, + 518, + 507, + 470, + 513, + 476, + 471, + 520, + 495, + 504, + 490, + 492, + 489, + 467, + 480, + 463, + 489, + 499, + 479, + 507, + 510, + 529, + 443, + 504, + 483, + 484, + 452, + 459, + 486, + 481, + 475, + 501, + 474, + 485, + 529, + 486, + 469, + 451, + 484, + 499, + 484, + 483, + 504, + 507, + 471, + 517, + 508, + 497, + 444, + 484, + 474, + 508, + 508, + 518, + 489, + 479, + 498, + 484, + 502, + 485, + 537, + 464, + 458, + 471, + 500, + 494, + 486, + 513, + 490, + 521, + 450, + 482, + 473, + 479, + 478, + 461, + 485, + 480, + 489, + 471, + 521, + 524, + 482, + 514, + 477, + 550, + 486, + 532, + 514, + 497, + 499, + 491, + 489, + 500, + 496, + 484, + 492, + 460, + 480, + 483, + 482, + 512, + 460, + 453, + 515, + 477, + 478, + 502, + 524, + 498, + 494, + 525, + 481, + 458, + 493, + 473, + 511, + 469, + 477, + 495, + 445, + 476, + 494, + 504, + 488, + 481, + 480, + 479, + 502, + 482, + 512, + 485, + 502, + 501, + 467, + 448, + 497, + 509, + 507, + 471, + 478, + 483, + 504, + 513, + 436, + 440, + 507, + 491, + 504, + 507, + 466, + 488, + 480, + 506, + 479, + 486, + 482, + 480, + 497, + 502, + 457, + 489, + 470, + 507, + 480, + 481, + 458, + 502, + 485, + 498, + 461, + 531, + 547, + 464, + 465, + 510, + 461, + 461, + 474, + 529, + 499, + 503, + 477, + 537, + 509, + 481, + 539, + 528, + 508, + 475, + 476, + 487, + 504, + 495, + 484, + 486, + 501, + 462, + 501, + 465, + 506, + 478, + 472, + 545, + 487, + 464, + 463, + 516, + 474, + 520, + 462, + 502, + 500, + 453, + 477, + 476, + 477, + 504, + 482, + 488, + 506, + 472, + 500, + 500, + 507, + 497, + 498, + 465, + 516, + 457, + 474, + 464, + 451, + 522, + 484, + 429, + 508, + 449, + 467, + 488, + 503, + 483, + 488, + 492, + 434, + 472, + 522, + 486, + 520, + 507, + 517, + 499, + 481, + 445, + 465, + 516, + 455, + 520, + 438, + 467, + 489, + 498, + 500, + 487, + 505, + 477, + 487, + 480, + 491, + 515, + 496, + 462, + 456, + 479, + 498, + 494, + 478, + 476, + 476, + 478, + 467, + 530, + 477, + 477, + 495, + 487, + 523, + 544, + 473, + 496, + 493, + 465, + 502, + 497, + 494, + 496, + 479, + 485, + 473, + 473, + 519, + 477, + 487, + 511, + 468, + 478, + 491, + 529, + 473, + 523, + 498, + 474, + 455, + 482, + 509, + 500, + 491, + 501, + 481, + 500, + 465, + 477, + 471, + 513, + 514, + 489, + 512, + 537, + 487, + 506, + 484, + 446, + 480, + 487, + 511, + 482, + 467, + 497, + 484, + 474, + 496, + 477, + 522, + 510, + 521, + 480, + 461, + 531, + 478, + 465, + 470, + 479, + 455, + 533, + 452, + 486, + 499, + 479, + 459, + 494, + 434, + 485, + 466, + 479, + 481, + 510, + 493, + 477, + 479, + 468, + 491, + 487, + 477, + 484, + 491, + 494, + 431, + 534, + 493, + 498, + 477, + 476, + 448, + 445, + 488, + 453, + 484, + 454, + 487, + 446, + 460, + 500, + 468, + 479, + 483, + 498, + 457, + 449, + 474, + 481, + 503, + 497, + 492, + 483, + 532, + 482, + 482, + 500, + 453, + 499, + 459, + 506, + 492, + 524, + 491, + 467, + 495, + 510, + 471, + 491, + 503, + 460, + 484, + 448, + 476, + 513, + 459, + 521, + 476, + 484, + 491, + 494, + 566, + 479, + 555, + 516, + 493, + 482, + 520, + 510, + 470, + 460, + 497, + 478, + 480, + 515, + 496, + 496, + 503, + 485, + 503, + 508, + 476, + 516, + 510, + 510, + 518, + 486, + 487, + 460, + 492, + 495, + 499, + 470, + 461, + 507, + 510, + 494, + 488, + 503, + 505, + 550, + 514, + 494, + 492, + 488, + 505, + 452, + 488, + 474, + 479, + 466, + 463, + 457, + 489, + 510, + 449, + 490, + 444, + 483, + 483, + 509, + 501, + 512, + 465, + 476, + 442, + 487, + 502, + 458, + 524, + 461, + 500, + 482, + 492, + 469, + 460, + 508, + 459, + 452, + 487, + 481, + 466, + 515, + 491, + 489, + 482, + 489, + 486, + 470, + 505, + 494, + 544, + 509, + 503, + 515, + 502, + 498, + 461, + 530, + 460, + 494, + 463, + 470, + 551, + 478, + 483, + 535, + 506, + 511, + 491, + 513, + 506, + 461, + 484, + 430, + 462, + 494, + 521, + 458, + 463, + 503, + 506, + 446, + 513, + 479, + 491, + 470, + 491, + 467, + 497, + 475, + 473, + 511, + 458, + 502, + 489, + 455, + 487, + 475, + 466, + 474, + 518, + 493, + 511, + 503, + 449, + 467, + 444, + 529, + 494, + 485, + 509, + 516, + 463, + 501, + 472, + 468, + 491, + 526, + 511, + 461, + 486, + 497, + 525, + 469, + 478, + 528, + 482, + 462, + 464, + 524, + 505, + 435, + 474, + 441, + 484, + 532, + 496, + 462, + 480, + 482, + 526, + 504, + 466, + 511, + 500, + 470, + 502, + 419, + 500, + 446, + 467, + 503, + 471, + 471, + 490, + 517, + 482, + 505, + 493, + 502, + 488, + 491, + 469, + 506, + 440, + 483, + 487, + 471, + 493, + 475, + 502, + 467, + 494, + 471, + 472, + 513, + 506, + 512, + 523, + 503, + 469, + 498, + 498, + 470, + 481, + 451, + 488, + 493, + 487, + 511, + 474, + 494, + 506, + 513, + 531, + 518, + 492, + 487, + 470, + 496, + 485, + 477, + 481, + 520, + 512, + 465, + 479, + 494, + 545, + 473, + 483, + 475, + 515, + 475, + 476, + 465, + 498, + 497, + 497, + 445, + 508, + 492, + 525, + 499, + 484, + 479, + 484, + 434, + 505, + 488, + 493, + 508, + 484, + 475, + 475, + 501, + 510, + 498, + 483, + 488, + 464, + 502, + 471, + 494, + 476, + 504, + 458, + 444, + 485, + 478, + 454, + 494, + 472, + 494, + 488, + 483, + 497, + 489, + 484, + 513, + 520, + 483, + 476, + 464, + 518, + 500, + 502, + 518, + 495, + 456, + 487, + 483, + 473, + 496, + 493, + 490, + 508, + 465, + 497, + 475, + 522, + 475, + 483, + 492, + 455, + 470, + 489, + 525, + 480, + 473, + 503, + 504, + 477, + 473, + 529, + 500, + 474, + 501, + 512, + 498, + 510, + 495, + 497, + 489, + 474, + 487, + 489, + 504, + 466, + 503, + 472, + 465, + 510, + 476, + 442, + 474, + 507, + 495, + 490, + 483, + 466, + 489, + 472, + 466, + 519, + 471, + 500, + 461, + 494, + 500, + 510, + 496, + 470, + 485, + 466, + 484, + 516, + 493, + 500, + 471, + 489, + 454, + 494, + 465, + 530, + 536, + 491, + 478, + 491, + 443, + 434, + 468, + 502, + 477, + 511, + 479, + 500, + 499, + 451, + 503, + 493, + 501, + 485, + 450, + 482, + 482, + 438, + 516, + 490, + 508, + 500, + 491, + 462, + 533, + 478, + 463, + 517, + 497, + 466, + 493, + 495, + 489, + 475, + 478, + 509, + 450, + 506, + 504, + 481, + 464, + 536, + 534, + 466, + 463, + 510, + 507, + 482, + 503, + 478, + 469, + 465, + 513, + 481, + 446, + 510, + 484, + 443, + 516, + 482, + 471, + 505, + 471, + 485, + 491, + 471, + 496, + 470, + 509, + 435, + 487, + 502, + 475, + 496, + 519, + 448, + 477, + 481, + 492, + 509, + 458, + 511, + 483, + 465, + 477, + 493, + 487, + 513, + 488, + 501, + 501, + 501, + 490, + 479, + 505, + 526, + 496, + 548, + 486, + 517, + 424, + 498, + 490, + 512, + 476, + 454, + 476, + 506, + 473, + 466, + 490, + 448, + 498, + 494, + 512, + 476, + 490, + 477, + 460, + 473, + 511, + 489, + 510, + 487, + 454, + 472, + 485, + 478, + 476, + 459, + 501, + 506, + 472, + 478, + 505, + 507, + 486, + 487, + 498, + 511, + 504, + 475, + 466, + 503, + 496, + 473, + 500, + 450, + 485, + 477, + 486, + 507, + 498, + 486, + 514, + 466, + 471, + 454, + 455, + 488, + 503, + 504, + 504, + 527, + 493, + 515, + 478, + 486, + 496, + 508, + 482, + 511, + 475, + 470, + 485, + 469, + 493, + 498, + 492, + 500, + 528, + 496, + 497, + 473, + 486, + 488, + 474, + 470, + 484, + 458, + 500, + 510, + 477, + 503, + 479, + 471, + 488, + 510, + 504, + 436, + 491, + 486, + 461, + 477, + 483, + 452, + 465, + 474, + 510, + 486, + 531, + 508, + 499, + 451, + 497, + 504, + 490, + 469, + 452, + 506, + 459, + 487, + 481, + 510, + 469, + 486, + 517, + 488, + 463, + 480, + 534, + 483, + 513, + 468, + 471, + 461, + 531, + 488, + 466, + 490, + 513, + 513, + 507, + 492, + 493, + 479, + 505, + 476, + 485, + 500, + 485, + 503, + 453, + 478, + 500, + 471, + 473, + 466, + 465, + 532, + 513, + 517, + 518, + 532, + 501, + 491, + 451, + 482, + 472, + 483, + 474, + 529, + 472, + 512, + 482, + 518, + 468, + 435, + 460, + 497, + 538, + 484, + 500, + 446, + 494, + 505, + 498, + 505, + 487, + 517, + 496, + 489, + 475, + 433, + 452, + 486, + 483, + 498, + 476, + 510, + 496, + 517, + 490, + 465, + 462, + 463, + 462, + 465, + 486, + 504, + 458, + 489, + 517, + 474, + 514, + 536, + 527, + 446, + 536, + 493, + 492, + 490, + 469, + 483, + 512, + 449, + 536, + 501, + 499, + 477, + 489, + 483, + 518, + 484, + 484, + 485, + 483, + 470, + 481, + 500, + 474, + 469, + 493, + 461, + 506, + 468, + 496, + 483, + 466, + 499, + 521, + 466, + 510, + 489, + 519, + 523, + 470, + 486, + 489, + 536, + 499, + 501, + 489, + 501, + 473, + 485, + 491, + 514, + 508, + 495, + 457, + 479, + 484, + 474, + 496, + 481, + 515, + 468, + 480, + 482, + 519, + 495, + 496, + 511, + 464, + 511, + 483, + 441, + 494, + 486, + 512, + 434, + 501, + 509, + 430, + 473, + 529, + 487, + 485, + 512, + 460, + 513, + 436, + 487, + 493, + 499, + 491, + 469, + 478, + 491, + 499, + 469, + 484, + 459, + 442, + 441, + 496, + 481, + 540, + 506, + 470, + 516, + 507, + 466, + 461, + 534, + 514, + 469, + 516, + 453, + 436, + 468, + 478, + 468, + 454, + 464, + 505, + 510, + 532, + 497, + 495, + 504, + 482, + 500, + 497, + 488, + 470, + 502, + 490, + 502, + 466, + 461, + 494, + 491, + 511, + 531, + 465, + 532, + 474, + 499, + 496, + 481, + 486, + 473, + 496, + 494, + 501, + 495, + 477, + 509, + 525, + 510, + 474, + 514, + 493, + 510, + 513, + 505, + 475, + 493, + 458, + 462, + 467, + 468, + 474, + 501, + 539, + 509, + 480, + 500, + 480, + 468, + 464, + 500, + 457, + 497, + 467, + 472, + 528, + 473, + 472, + 528, + 488, + 482, + 490, + 502, + 523, + 469, + 475, + 493, + 473, + 455, + 461, + 469, + 480, + 460, + 497, + 478, + 509, + 495, + 491, + 523, + 508, + 495, + 493, + 460, + 460, + 495, + 483, + 498, + 494, + 501, + 471, + 495, + 465, + 476, + 476, + 502, + 495, + 503, + 491, + 454, + 458, + 511, + 497, + 541, + 468, + 460, + 499, + 495, + 481, + 467, + 453, + 512, + 451, + 502, + 481, + 495, + 456, + 481, + 484, + 459, + 520, + 470, + 480, + 512, + 510, + 472, + 446, + 469, + 496, + 488, + 493, + 503, + 513, + 514, + 500, + 495, + 501, + 484, + 466, + 504, + 494, + 451, + 477, + 494, + 494, + 471, + 481, + 514, + 458, + 488, + 507, + 489, + 465, + 475, + 478, + 496, + 462, + 465, + 499, + 488, + 502, + 507, + 448, + 460, + 480, + 486, + 466, + 500, + 528, + 486, + 475, + 475, + 493, + 458, + 475, + 488, + 513, + 493, + 468, + 508, + 454, + 533, + 474, + 445, + 481, + 492, + 522, + 506, + 514, + 496, + 470, + 509, + 504, + 512, + 498, + 519, + 531, + 499, + 474, + 494, + 497, + 515, + 506, + 506, + 515, + 497, + 472, + 501, + 514, + 490, + 468, + 483, + 491, + 532, + 494, + 533, + 506, + 491, + 446, + 497, + 469, + 478, + 478, + 469, + 467, + 496, + 518, + 456, + 504, + 472, + 469, + 486, + 486, + 506, + 440, + 473, + 475, + 459, + 496, + 506, + 517, + 483, + 484, + 507, + 481, + 467, + 483, + 481, + 504, + 446, + 467, + 476, + 477, + 447, + 489, + 494, + 474, + 511, + 500, + 486, + 509, + 499, + 488, + 480, + 503, + 484, + 485, + 503, + 483, + 483, + 483, + 447, + 491, + 502, + 492, + 494, + 491, + 465, + 503, + 499, + 484, + 461, + 497, + 482, + 477, + 468, + 511, + 481, + 485, + 470, + 490, + 515, + 526, + 457, + 510, + 476, + 501, + 437, + 496, + 531, + 495, + 472, + 485, + 471, + 517, + 518, + 519, + 459, + 490, + 522, + 476, + 465, + 466, + 508, + 477, + 482, + 503, + 476, + 486, + 473, + 499, + 493, + 490, + 492, + 478, + 522, + 523, + 494, + 512, + 482, + 502, + 474, + 474, + 482, + 486, + 502, + 462, + 536, + 486, + 484, + 474, + 503, + 506, + 508, + 506, + 500, + 468, + 494, + 462, + 528, + 451, + 501, + 457, + 476, + 481, + 493, + 479, + 467, + 498, + 508, + 520, + 492, + 447, + 508, + 491, + 490, + 448, + 466, + 474, + 497, + 469, + 521, + 538, + 501, + 501, + 429, + 510, + 499, + 509, + 487, + 493, + 518, + 495, + 482, + 494, + 502, + 529, + 460, + 553, + 506, + 490, + 534, + 453, + 500, + 491, + 521, + 457, + 452, + 520, + 517, + 456, + 476, + 474, + 495, + 456, + 496, + 516, + 490, + 485, + 463, + 487, + 453, + 466, + 486, + 459, + 485, + 484, + 491, + 463, + 499, + 511, + 454, + 488, + 461, + 507, + 493, + 507, + 499, + 505, + 461, + 496, + 546, + 492, + 500, + 437, + 493, + 495, + 484, + 453, + 465, + 480, + 477, + 508, + 450, + 475, + 461, + 480, + 486, + 491, + 476, + 534, + 515, + 475, + 486, + 539, + 462, + 445, + 529, + 490, + 470, + 463, + 520, + 493, + 496, + 469, + 487, + 488, + 519, + 498, + 534, + 522, + 470, + 494, + 494, + 497, + 537, + 498, + 484, + 468, + 475, + 503, + 515, + 450, + 460, + 479, + 487, + 467, + 466, + 460, + 504, + 492, + 516, + 503, + 465, + 480, + 487, + 477, + 484, + 473, + 470, + 526, + 471, + 460, + 500, + 479, + 484, + 483, + 490, + 522, + 475, + 500, + 486, + 490, + 485, + 434, + 480 + ] + }, + { + "counters": [ + 489, + 519, + 466, + 483, + 484, + 514, + 505, + 495, + 485, + 472, + 525, + 511, + 497, + 486, + 478, + 481, + 483, + 502, + 468, + 473, + 546, + 482, + 497, + 456, + 471, + 501, + 479, + 480, + 489, + 513, + 499, + 481, + 507, + 472, + 503, + 467, + 503, + 535, + 502, + 492, + 518, + 487, + 482, + 469, + 515, + 488, + 475, + 512, + 514, + 491, + 478, + 485, + 478, + 476, + 499, + 447, + 491, + 534, + 493, + 491, + 471, + 465, + 500, + 455, + 505, + 484, + 461, + 480, + 497, + 463, + 497, + 488, + 467, + 507, + 504, + 465, + 503, + 492, + 503, + 486, + 497, + 494, + 527, + 485, + 478, + 514, + 471, + 477, + 493, + 502, + 493, + 489, + 505, + 504, + 483, + 508, + 473, + 503, + 520, + 520, + 460, + 467, + 509, + 490, + 467, + 466, + 497, + 484, + 490, + 494, + 498, + 453, + 454, + 469, + 529, + 462, + 491, + 513, + 499, + 520, + 456, + 494, + 507, + 515, + 504, + 474, + 489, + 495, + 483, + 467, + 468, + 479, + 510, + 488, + 461, + 484, + 482, + 492, + 482, + 468, + 497, + 554, + 503, + 457, + 496, + 483, + 464, + 447, + 446, + 518, + 449, + 531, + 511, + 494, + 497, + 512, + 483, + 486, + 476, + 499, + 506, + 481, + 511, + 505, + 518, + 506, + 506, + 508, + 501, + 502, + 511, + 493, + 464, + 515, + 488, + 494, + 488, + 513, + 503, + 440, + 451, + 495, + 451, + 437, + 516, + 495, + 524, + 493, + 485, + 490, + 464, + 504, + 456, + 481, + 517, + 486, + 485, + 471, + 506, + 492, + 531, + 468, + 529, + 495, + 486, + 512, + 494, + 484, + 488, + 476, + 486, + 499, + 504, + 452, + 488, + 469, + 467, + 479, + 474, + 446, + 473, + 473, + 465, + 501, + 487, + 499, + 488, + 487, + 502, + 514, + 469, + 483, + 487, + 469, + 490, + 486, + 446, + 445, + 509, + 480, + 500, + 478, + 490, + 496, + 514, + 464, + 512, + 468, + 480, + 503, + 530, + 494, + 488, + 493, + 535, + 508, + 483, + 475, + 488, + 478, + 483, + 491, + 455, + 509, + 473, + 500, + 514, + 458, + 488, + 473, + 524, + 496, + 477, + 477, + 497, + 512, + 476, + 435, + 474, + 497, + 513, + 500, + 496, + 475, + 464, + 514, + 496, + 528, + 501, + 476, + 489, + 456, + 455, + 481, + 498, + 456, + 489, + 455, + 469, + 477, + 466, + 491, + 485, + 466, + 490, + 462, + 456, + 470, + 479, + 494, + 495, + 514, + 514, + 495, + 478, + 457, + 456, + 472, + 496, + 514, + 501, + 500, + 486, + 508, + 528, + 472, + 488, + 475, + 475, + 458, + 437, + 502, + 468, + 464, + 498, + 511, + 496, + 504, + 457, + 489, + 494, + 478, + 466, + 479, + 482, + 497, + 497, + 472, + 456, + 540, + 489, + 484, + 470, + 514, + 498, + 492, + 493, + 481, + 471, + 477, + 477, + 472, + 514, + 498, + 496, + 473, + 498, + 510, + 528, + 502, + 467, + 502, + 492, + 490, + 465, + 513, + 504, + 500, + 498, + 508, + 501, + 499, + 499, + 501, + 459, + 491, + 491, + 494, + 504, + 464, + 506, + 468, + 499, + 499, + 492, + 474, + 484, + 464, + 492, + 451, + 490, + 500, + 465, + 472, + 526, + 447, + 498, + 485, + 495, + 489, + 505, + 479, + 452, + 470, + 512, + 496, + 489, + 486, + 474, + 510, + 477, + 468, + 467, + 496, + 448, + 469, + 506, + 443, + 505, + 488, + 461, + 493, + 464, + 510, + 504, + 476, + 504, + 520, + 468, + 495, + 500, + 499, + 487, + 469, + 481, + 480, + 468, + 500, + 455, + 479, + 449, + 459, + 452, + 485, + 462, + 533, + 487, + 493, + 501, + 513, + 473, + 473, + 486, + 463, + 499, + 463, + 463, + 463, + 513, + 474, + 508, + 533, + 459, + 494, + 496, + 455, + 490, + 490, + 480, + 477, + 474, + 463, + 508, + 493, + 502, + 492, + 493, + 490, + 510, + 497, + 480, + 502, + 456, + 479, + 458, + 477, + 499, + 500, + 457, + 451, + 459, + 475, + 495, + 471, + 482, + 472, + 473, + 489, + 488, + 445, + 462, + 504, + 496, + 453, + 503, + 484, + 488, + 491, + 501, + 481, + 477, + 485, + 492, + 507, + 469, + 495, + 525, + 532, + 452, + 482, + 520, + 504, + 456, + 509, + 471, + 503, + 504, + 476, + 488, + 445, + 471, + 501, + 477, + 502, + 503, + 481, + 487, + 465, + 459, + 500, + 459, + 493, + 504, + 503, + 499, + 500, + 472, + 547, + 503, + 463, + 463, + 489, + 482, + 534, + 487, + 484, + 456, + 502, + 481, + 507, + 482, + 502, + 491, + 473, + 477, + 477, + 471, + 516, + 471, + 496, + 469, + 479, + 500, + 473, + 479, + 492, + 500, + 463, + 487, + 499, + 481, + 494, + 489, + 497, + 469, + 457, + 493, + 462, + 488, + 493, + 527, + 508, + 502, + 462, + 473, + 457, + 453, + 498, + 483, + 475, + 466, + 480, + 500, + 499, + 491, + 453, + 487, + 458, + 477, + 504, + 492, + 521, + 473, + 485, + 424, + 471, + 472, + 445, + 491, + 488, + 484, + 435, + 490, + 513, + 486, + 476, + 499, + 495, + 446, + 509, + 481, + 473, + 488, + 490, + 488, + 482, + 506, + 481, + 497, + 503, + 519, + 522, + 486, + 518, + 480, + 492, + 507, + 479, + 524, + 512, + 498, + 511, + 490, + 527, + 519, + 475, + 469, + 537, + 448, + 466, + 535, + 481, + 522, + 514, + 485, + 488, + 492, + 508, + 485, + 497, + 481, + 495, + 502, + 472, + 487, + 448, + 464, + 485, + 566, + 514, + 430, + 491, + 469, + 425, + 512, + 497, + 528, + 498, + 466, + 477, + 505, + 483, + 498, + 477, + 455, + 491, + 526, + 468, + 464, + 504, + 502, + 491, + 488, + 482, + 477, + 452, + 432, + 459, + 521, + 422, + 513, + 488, + 470, + 519, + 522, + 471, + 503, + 496, + 502, + 496, + 548, + 497, + 504, + 486, + 488, + 505, + 511, + 474, + 466, + 489, + 457, + 492, + 509, + 483, + 509, + 464, + 478, + 492, + 511, + 476, + 458, + 465, + 485, + 482, + 463, + 485, + 492, + 470, + 479, + 472, + 463, + 512, + 484, + 498, + 487, + 485, + 468, + 493, + 455, + 483, + 474, + 497, + 484, + 452, + 472, + 515, + 461, + 460, + 503, + 503, + 449, + 479, + 435, + 525, + 478, + 453, + 472, + 462, + 484, + 492, + 475, + 525, + 498, + 432, + 488, + 461, + 483, + 502, + 530, + 497, + 477, + 483, + 497, + 495, + 478, + 490, + 471, + 476, + 507, + 500, + 466, + 511, + 484, + 513, + 487, + 514, + 466, + 478, + 510, + 505, + 505, + 502, + 469, + 515, + 495, + 506, + 493, + 511, + 469, + 472, + 512, + 471, + 486, + 490, + 492, + 468, + 515, + 531, + 489, + 516, + 468, + 515, + 488, + 462, + 470, + 456, + 522, + 467, + 457, + 479, + 478, + 521, + 519, + 520, + 436, + 427, + 503, + 514, + 518, + 512, + 492, + 454, + 489, + 492, + 506, + 499, + 502, + 470, + 468, + 456, + 498, + 448, + 502, + 517, + 509, + 482, + 519, + 525, + 501, + 514, + 478, + 485, + 504, + 502, + 481, + 484, + 525, + 502, + 497, + 497, + 485, + 492, + 452, + 421, + 449, + 502, + 517, + 474, + 479, + 478, + 493, + 483, + 459, + 441, + 465, + 509, + 473, + 455, + 518, + 433, + 473, + 507, + 498, + 472, + 489, + 504, + 482, + 490, + 491, + 533, + 487, + 480, + 504, + 526, + 519, + 496, + 468, + 466, + 493, + 518, + 528, + 501, + 480, + 479, + 472, + 481, + 480, + 507, + 457, + 519, + 507, + 468, + 518, + 464, + 492, + 478, + 473, + 539, + 466, + 515, + 465, + 489, + 461, + 476, + 457, + 482, + 491, + 491, + 474, + 464, + 478, + 519, + 487, + 503, + 482, + 474, + 468, + 459, + 432, + 474, + 469, + 493, + 485, + 504, + 486, + 506, + 492, + 489, + 489, + 530, + 503, + 499, + 509, + 499, + 488, + 514, + 440, + 545, + 487, + 452, + 500, + 509, + 485, + 517, + 496, + 513, + 513, + 483, + 494, + 514, + 474, + 502, + 509, + 524, + 520, + 444, + 475, + 488, + 505, + 493, + 489, + 474, + 451, + 487, + 490, + 499, + 481, + 494, + 465, + 454, + 479, + 487, + 486, + 477, + 495, + 501, + 471, + 479, + 469, + 464, + 502, + 486, + 439, + 478, + 501, + 505, + 444, + 505, + 481, + 514, + 494, + 463, + 472, + 473, + 472, + 452, + 488, + 526, + 494, + 476, + 522, + 514, + 502, + 479, + 487, + 496, + 483, + 481, + 481, + 476, + 514, + 511, + 484, + 457, + 502, + 539, + 489, + 464, + 506, + 465, + 446, + 528, + 459, + 455, + 478, + 520, + 472, + 440, + 490, + 533, + 487, + 494, + 492, + 488, + 526, + 521, + 492, + 485, + 481, + 439, + 459, + 495, + 472, + 477, + 490, + 511, + 452, + 511, + 480, + 496, + 527, + 483, + 479, + 479, + 476, + 481, + 462, + 515, + 440, + 487, + 498, + 484, + 468, + 475, + 518, + 486, + 508, + 499, + 465, + 487, + 497, + 467, + 484, + 533, + 505, + 491, + 520, + 459, + 495, + 527, + 463, + 480, + 473, + 542, + 503, + 441, + 472, + 462, + 486, + 496, + 516, + 462, + 537, + 511, + 499, + 493, + 495, + 447, + 489, + 517, + 435, + 476, + 492, + 475, + 485, + 493, + 471, + 496, + 514, + 492, + 477, + 500, + 518, + 488, + 539, + 468, + 536, + 471, + 491, + 488, + 507, + 511, + 496, + 450, + 486, + 520, + 514, + 474, + 477, + 464, + 472, + 474, + 464, + 496, + 462, + 457, + 439, + 510, + 462, + 469, + 508, + 459, + 494, + 509, + 466, + 511, + 523, + 466, + 495, + 454, + 512, + 469, + 467, + 478, + 453, + 520, + 491, + 483, + 493, + 500, + 444, + 492, + 473, + 502, + 455, + 494, + 549, + 508, + 466, + 494, + 479, + 508, + 473, + 463, + 483, + 471, + 525, + 517, + 479, + 500, + 500, + 473, + 478, + 496, + 444, + 500, + 495, + 513, + 507, + 491, + 453, + 498, + 464, + 476, + 523, + 460, + 492, + 525, + 517, + 483, + 480, + 485, + 465, + 512, + 490, + 506, + 547, + 486, + 462, + 484, + 524, + 476, + 471, + 479, + 496, + 546, + 489, + 473, + 468, + 492, + 490, + 494, + 481, + 488, + 457, + 520, + 495, + 492, + 476, + 471, + 494, + 518, + 519, + 502, + 510, + 499, + 507, + 477, + 500, + 491, + 492, + 500, + 496, + 490, + 519, + 474, + 434, + 482, + 482, + 501, + 486, + 456, + 510, + 502, + 488, + 503, + 473, + 521, + 517, + 464, + 506, + 462, + 471, + 497, + 472, + 495, + 507, + 481, + 478, + 477, + 421, + 528, + 509, + 534, + 511, + 513, + 487, + 536, + 502, + 532, + 512, + 461, + 480, + 499, + 456, + 504, + 478, + 496, + 485, + 467, + 507, + 491, + 482, + 483, + 479, + 478, + 523, + 469, + 516, + 497, + 500, + 478, + 461, + 503, + 480, + 480, + 468, + 507, + 473, + 500, + 503, + 488, + 486, + 497, + 481, + 523, + 482, + 488, + 489, + 517, + 493, + 528, + 489, + 481, + 510, + 470, + 467, + 490, + 476, + 489, + 465, + 482, + 487, + 498, + 497, + 515, + 470, + 487, + 472, + 522, + 468, + 495, + 477, + 438, + 474, + 532, + 525, + 452, + 495, + 474, + 496, + 524, + 473, + 499, + 519, + 505, + 507, + 484, + 434, + 484, + 490, + 537, + 519, + 462, + 484, + 479, + 457, + 460, + 512, + 487, + 503, + 502, + 463, + 489, + 497, + 494, + 506, + 495, + 518, + 473, + 446, + 486, + 462, + 515, + 473, + 502, + 514, + 518, + 476, + 498, + 455, + 517, + 492, + 424, + 490, + 485, + 502, + 499, + 479, + 468, + 460, + 450, + 464, + 482, + 478, + 494, + 530, + 498, + 475, + 483, + 505, + 482, + 500, + 470, + 496, + 458, + 460, + 524, + 435, + 490, + 494, + 516, + 491, + 487, + 482, + 480, + 498, + 487, + 479, + 503, + 470, + 477, + 468, + 500, + 489, + 447, + 495, + 490, + 501, + 494, + 516, + 475, + 461, + 510, + 489, + 467, + 499, + 485, + 483, + 520, + 486, + 488, + 515, + 494, + 516, + 517, + 509, + 503, + 475, + 511, + 449, + 510, + 497, + 506, + 474, + 506, + 516, + 534, + 487, + 485, + 455, + 487, + 499, + 506, + 481, + 496, + 505, + 497, + 504, + 502, + 488, + 496, + 483, + 521, + 528, + 479, + 471, + 498, + 504, + 491, + 476, + 482, + 472, + 486, + 478, + 480, + 534, + 463, + 494, + 487, + 497, + 473, + 496, + 465, + 515, + 498, + 453, + 516, + 474, + 463, + 502, + 516, + 496, + 475, + 503, + 478, + 510, + 479, + 518, + 479, + 519, + 499, + 464, + 475, + 469, + 480, + 483, + 492, + 468, + 507, + 474, + 452, + 509, + 439, + 478, + 510, + 521, + 522, + 498, + 525, + 499, + 489, + 485, + 514, + 501, + 506, + 466, + 462, + 457, + 490, + 538, + 462, + 489, + 451, + 499, + 481, + 468, + 509, + 508, + 517, + 533, + 469, + 497, + 513, + 511, + 523, + 482, + 492, + 507, + 512, + 477, + 487, + 490, + 495, + 462, + 491, + 505, + 500, + 484, + 475, + 515, + 490, + 471, + 547, + 480, + 501, + 483, + 469, + 476, + 513, + 506, + 494, + 462, + 494, + 490, + 466, + 493, + 509, + 476, + 492, + 483, + 442, + 512, + 475, + 502, + 521, + 467, + 469, + 458, + 476, + 467, + 474, + 517, + 485, + 502, + 507, + 473, + 488, + 464, + 483, + 469, + 537, + 464, + 509, + 498, + 489, + 471, + 479, + 520, + 525, + 490, + 454, + 519, + 475, + 472, + 475, + 517, + 513, + 502, + 493, + 485, + 524, + 488, + 511, + 483, + 496, + 502, + 462, + 485, + 505, + 469, + 526, + 484, + 523, + 482, + 477, + 470, + 485, + 469, + 480, + 509, + 485, + 486, + 524, + 510, + 503, + 457, + 464, + 492, + 473, + 480, + 503, + 493, + 517, + 491, + 471, + 480, + 530, + 457, + 469, + 494, + 500, + 489, + 506, + 462, + 455, + 494, + 472, + 481, + 481, + 527, + 523, + 440, + 502, + 447, + 469, + 479, + 515, + 495, + 522, + 494, + 513, + 537, + 507, + 501, + 506, + 438, + 501, + 486, + 441, + 498, + 486, + 498, + 498, + 513, + 521, + 502, + 474, + 519, + 477, + 521, + 492, + 505, + 488, + 485, + 519, + 457, + 492, + 470, + 484, + 473, + 499, + 516, + 479, + 506, + 476, + 455, + 472, + 476, + 471, + 490, + 482, + 517, + 463, + 466, + 485, + 502, + 486, + 472, + 469, + 502, + 537, + 452, + 528, + 485, + 497, + 472, + 530, + 485, + 458, + 477, + 547, + 519, + 485, + 466, + 463, + 534, + 512, + 491, + 483, + 502, + 512, + 478, + 482, + 496, + 476, + 483, + 487, + 484, + 473, + 526, + 485, + 440, + 498, + 507, + 499, + 493, + 480, + 471, + 494, + 496, + 477, + 490, + 465, + 451, + 508, + 469, + 427, + 500, + 484, + 493, + 493, + 478, + 468, + 508, + 539, + 508, + 481, + 517, + 485, + 469, + 475, + 494, + 508, + 473, + 481, + 502, + 481, + 470, + 493, + 485, + 471, + 505, + 495, + 487, + 461, + 520, + 500, + 469, + 488, + 489, + 495, + 480, + 479, + 461, + 452, + 487, + 501, + 490, + 474, + 488, + 490, + 517, + 506, + 496, + 510, + 542, + 482, + 474, + 490, + 492, + 475, + 496, + 479, + 485, + 503, + 511, + 474, + 504, + 482, + 518, + 517, + 483, + 479, + 493, + 445, + 477, + 497, + 491, + 505, + 488, + 458, + 511, + 517, + 498, + 490, + 467, + 528, + 466, + 481, + 487, + 484, + 503, + 459, + 474, + 503, + 476, + 494, + 453, + 482, + 512, + 490, + 467, + 509, + 475, + 469, + 463, + 494, + 447, + 478, + 507, + 520, + 497, + 502, + 564, + 456, + 466, + 513, + 507, + 498, + 526, + 464, + 496, + 495, + 486, + 462, + 469, + 506, + 494, + 500, + 506, + 457, + 515, + 497, + 471, + 491, + 483, + 476, + 520, + 479, + 473, + 493, + 481, + 447, + 502, + 541, + 444, + 467, + 459, + 497, + 502, + 490, + 465, + 502, + 463, + 480, + 473, + 457, + 470, + 487, + 529, + 494, + 467, + 495, + 488, + 479, + 484, + 533, + 485, + 458, + 490, + 502, + 473, + 493, + 473, + 480, + 494, + 514, + 503, + 470, + 454, + 496, + 500, + 506, + 461, + 498, + 489, + 484, + 458, + 474, + 472, + 488, + 481, + 517, + 511, + 485, + 518, + 499, + 485, + 478, + 498, + 502, + 483, + 468, + 463, + 455, + 499, + 491 + ] + }, + { + "counters": [ + 529, + 495, + 464, + 493, + 511, + 495, + 481, + 475, + 475, + 471, + 481, + 520, + 471, + 499, + 479, + 524, + 517, + 483, + 474, + 486, + 530, + 516, + 499, + 464, + 501, + 540, + 452, + 486, + 496, + 495, + 510, + 517, + 483, + 507, + 485, + 466, + 496, + 436, + 506, + 501, + 484, + 479, + 472, + 486, + 441, + 515, + 515, + 479, + 482, + 499, + 483, + 487, + 475, + 507, + 504, + 495, + 485, + 466, + 534, + 489, + 533, + 475, + 511, + 492, + 489, + 502, + 465, + 486, + 462, + 463, + 520, + 508, + 488, + 477, + 478, + 485, + 478, + 515, + 507, + 513, + 522, + 478, + 452, + 482, + 470, + 464, + 492, + 459, + 465, + 491, + 494, + 507, + 489, + 470, + 448, + 455, + 531, + 490, + 516, + 498, + 488, + 483, + 469, + 521, + 476, + 516, + 478, + 468, + 489, + 456, + 492, + 485, + 513, + 481, + 503, + 488, + 492, + 492, + 484, + 497, + 479, + 472, + 457, + 466, + 478, + 482, + 468, + 468, + 460, + 475, + 495, + 479, + 481, + 464, + 496, + 466, + 468, + 474, + 487, + 487, + 477, + 513, + 468, + 473, + 463, + 523, + 480, + 454, + 479, + 439, + 492, + 474, + 468, + 502, + 489, + 515, + 490, + 485, + 456, + 461, + 506, + 491, + 457, + 471, + 499, + 473, + 526, + 506, + 514, + 466, + 454, + 470, + 494, + 500, + 487, + 485, + 508, + 459, + 518, + 498, + 496, + 514, + 463, + 496, + 466, + 461, + 507, + 493, + 481, + 479, + 491, + 508, + 485, + 472, + 524, + 482, + 473, + 504, + 495, + 531, + 509, + 483, + 490, + 462, + 505, + 469, + 484, + 446, + 502, + 484, + 496, + 463, + 485, + 442, + 479, + 518, + 510, + 463, + 452, + 488, + 516, + 514, + 507, + 448, + 501, + 496, + 484, + 451, + 543, + 535, + 509, + 532, + 474, + 457, + 504, + 513, + 503, + 460, + 456, + 468, + 475, + 509, + 496, + 465, + 501, + 526, + 481, + 494, + 472, + 513, + 503, + 505, + 451, + 491, + 513, + 508, + 504, + 479, + 475, + 477, + 469, + 512, + 518, + 492, + 514, + 458, + 497, + 484, + 509, + 493, + 500, + 474, + 470, + 496, + 468, + 495, + 491, + 504, + 505, + 460, + 435, + 489, + 459, + 478, + 505, + 538, + 485, + 465, + 511, + 473, + 480, + 459, + 495, + 480, + 475, + 476, + 482, + 500, + 499, + 490, + 470, + 455, + 502, + 448, + 491, + 450, + 457, + 484, + 487, + 503, + 523, + 454, + 501, + 506, + 483, + 492, + 468, + 464, + 483, + 486, + 491, + 483, + 466, + 480, + 482, + 498, + 497, + 512, + 464, + 526, + 533, + 471, + 481, + 453, + 484, + 516, + 521, + 503, + 501, + 509, + 491, + 477, + 481, + 516, + 466, + 466, + 463, + 503, + 519, + 503, + 493, + 469, + 484, + 477, + 449, + 454, + 488, + 482, + 511, + 466, + 467, + 488, + 455, + 491, + 463, + 479, + 486, + 508, + 485, + 473, + 482, + 445, + 457, + 487, + 466, + 486, + 505, + 490, + 483, + 490, + 468, + 466, + 519, + 498, + 508, + 499, + 446, + 478, + 505, + 495, + 472, + 502, + 465, + 470, + 481, + 462, + 502, + 499, + 469, + 505, + 513, + 485, + 504, + 491, + 470, + 467, + 488, + 503, + 499, + 477, + 510, + 458, + 518, + 484, + 481, + 531, + 482, + 523, + 470, + 509, + 491, + 496, + 491, + 477, + 443, + 484, + 486, + 463, + 485, + 479, + 495, + 453, + 480, + 504, + 484, + 458, + 498, + 498, + 487, + 536, + 499, + 496, + 516, + 480, + 492, + 470, + 492, + 446, + 524, + 484, + 493, + 508, + 486, + 535, + 494, + 477, + 504, + 505, + 473, + 472, + 523, + 501, + 507, + 514, + 454, + 513, + 501, + 502, + 490, + 477, + 476, + 452, + 477, + 483, + 481, + 464, + 504, + 476, + 500, + 511, + 501, + 470, + 470, + 496, + 502, + 475, + 502, + 504, + 480, + 497, + 475, + 486, + 491, + 508, + 523, + 519, + 479, + 473, + 452, + 450, + 479, + 487, + 512, + 490, + 503, + 482, + 492, + 486, + 464, + 500, + 488, + 538, + 517, + 487, + 497, + 442, + 484, + 487, + 523, + 507, + 460, + 459, + 503, + 508, + 459, + 515, + 515, + 506, + 502, + 468, + 489, + 482, + 461, + 524, + 473, + 465, + 500, + 513, + 474, + 496, + 476, + 490, + 505, + 517, + 449, + 495, + 494, + 523, + 492, + 484, + 460, + 505, + 499, + 494, + 498, + 460, + 496, + 496, + 488, + 463, + 523, + 490, + 509, + 522, + 499, + 477, + 486, + 460, + 468, + 463, + 494, + 509, + 482, + 489, + 477, + 497, + 498, + 518, + 450, + 444, + 486, + 501, + 503, + 473, + 490, + 516, + 506, + 458, + 504, + 464, + 452, + 476, + 475, + 474, + 463, + 500, + 454, + 529, + 508, + 491, + 481, + 475, + 471, + 485, + 479, + 468, + 492, + 450, + 481, + 487, + 468, + 481, + 507, + 463, + 448, + 492, + 522, + 498, + 456, + 499, + 468, + 455, + 491, + 504, + 502, + 519, + 502, + 533, + 482, + 459, + 469, + 502, + 502, + 463, + 468, + 530, + 453, + 472, + 483, + 503, + 475, + 478, + 484, + 506, + 493, + 485, + 475, + 510, + 465, + 483, + 472, + 493, + 496, + 506, + 457, + 515, + 485, + 456, + 465, + 499, + 497, + 472, + 477, + 491, + 472, + 448, + 470, + 474, + 513, + 485, + 465, + 488, + 447, + 490, + 478, + 532, + 498, + 547, + 454, + 490, + 473, + 504, + 492, + 471, + 497, + 486, + 512, + 484, + 444, + 495, + 479, + 460, + 482, + 516, + 505, + 480, + 500, + 511, + 472, + 495, + 487, + 483, + 486, + 504, + 463, + 441, + 542, + 475, + 496, + 485, + 489, + 463, + 475, + 468, + 487, + 507, + 501, + 479, + 443, + 492, + 453, + 460, + 497, + 474, + 468, + 452, + 501, + 503, + 482, + 456, + 472, + 469, + 456, + 470, + 504, + 497, + 513, + 454, + 520, + 494, + 506, + 493, + 501, + 459, + 504, + 531, + 466, + 514, + 465, + 519, + 475, + 500, + 504, + 504, + 476, + 500, + 470, + 477, + 496, + 484, + 484, + 506, + 481, + 515, + 491, + 508, + 506, + 477, + 497, + 478, + 515, + 500, + 516, + 451, + 480, + 487, + 514, + 478, + 457, + 489, + 447, + 496, + 465, + 516, + 469, + 465, + 496, + 500, + 492, + 478, + 461, + 496, + 479, + 498, + 511, + 476, + 474, + 482, + 471, + 486, + 497, + 517, + 506, + 451, + 510, + 494, + 484, + 529, + 535, + 474, + 500, + 510, + 501, + 499, + 526, + 471, + 496, + 484, + 491, + 471, + 488, + 495, + 443, + 474, + 487, + 476, + 493, + 435, + 494, + 494, + 475, + 477, + 500, + 502, + 479, + 474, + 505, + 466, + 472, + 466, + 463, + 541, + 522, + 481, + 501, + 481, + 489, + 454, + 520, + 477, + 467, + 505, + 466, + 503, + 480, + 538, + 490, + 481, + 509, + 479, + 493, + 512, + 473, + 481, + 467, + 468, + 476, + 489, + 489, + 488, + 483, + 475, + 512, + 488, + 512, + 479, + 459, + 475, + 457, + 538, + 483, + 455, + 498, + 471, + 486, + 468, + 504, + 450, + 514, + 485, + 460, + 510, + 532, + 467, + 467, + 496, + 478, + 475, + 443, + 456, + 488, + 509, + 463, + 508, + 490, + 488, + 469, + 514, + 495, + 470, + 512, + 428, + 495, + 483, + 484, + 509, + 532, + 499, + 491, + 515, + 472, + 504, + 508, + 478, + 523, + 539, + 481, + 478, + 515, + 505, + 460, + 500, + 470, + 507, + 483, + 438, + 479, + 510, + 475, + 481, + 522, + 471, + 508, + 560, + 496, + 457, + 453, + 483, + 484, + 466, + 493, + 519, + 457, + 478, + 486, + 489, + 503, + 494, + 484, + 483, + 497, + 532, + 475, + 499, + 474, + 481, + 487, + 432, + 495, + 480, + 503, + 505, + 515, + 466, + 479, + 463, + 498, + 502, + 451, + 503, + 482, + 484, + 499, + 503, + 462, + 483, + 510, + 537, + 480, + 468, + 508, + 490, + 514, + 451, + 507, + 517, + 517, + 478, + 461, + 436, + 502, + 511, + 503, + 475, + 513, + 485, + 462, + 489, + 476, + 500, + 496, + 497, + 470, + 501, + 492, + 455, + 465, + 528, + 468, + 519, + 463, + 516, + 501, + 476, + 491, + 444, + 480, + 499, + 491, + 472, + 510, + 484, + 497, + 494, + 479, + 519, + 505, + 527, + 506, + 488, + 494, + 499, + 514, + 468, + 476, + 477, + 513, + 496, + 471, + 532, + 486, + 469, + 488, + 527, + 489, + 473, + 475, + 504, + 485, + 520, + 479, + 472, + 508, + 475, + 493, + 498, + 464, + 446, + 471, + 463, + 471, + 500, + 491, + 490, + 543, + 500, + 513, + 521, + 473, + 481, + 467, + 425, + 461, + 476, + 455, + 472, + 510, + 489, + 469, + 510, + 499, + 489, + 469, + 495, + 488, + 514, + 467, + 525, + 497, + 512, + 517, + 500, + 479, + 461, + 479, + 517, + 522, + 491, + 478, + 508, + 498, + 458, + 504, + 480, + 460, + 481, + 527, + 528, + 484, + 474, + 481, + 462, + 462, + 483, + 497, + 486, + 494, + 535, + 466, + 441, + 495, + 489, + 469, + 497, + 510, + 519, + 475, + 483, + 503, + 473, + 509, + 511, + 487, + 464, + 495, + 468, + 501, + 498, + 509, + 502, + 480, + 488, + 511, + 451, + 483, + 518, + 484, + 530, + 488, + 543, + 487, + 476, + 474, + 479, + 519, + 514, + 500, + 493, + 506, + 469, + 457, + 514, + 469, + 475, + 476, + 490, + 505, + 472, + 511, + 499, + 469, + 482, + 456, + 510, + 475, + 483, + 500, + 453, + 504, + 483, + 498, + 457, + 481, + 507, + 493, + 459, + 480, + 463, + 479, + 475, + 454, + 517, + 484, + 473, + 464, + 460, + 505, + 515, + 483, + 520, + 504, + 466, + 497, + 491, + 490, + 500, + 500, + 487, + 493, + 467, + 501, + 463, + 501, + 450, + 457, + 503, + 486, + 468, + 483, + 482, + 494, + 497, + 468, + 483, + 447, + 454, + 457, + 474, + 461, + 473, + 500, + 463, + 496, + 510, + 475, + 506, + 475, + 468, + 488, + 504, + 475, + 479, + 462, + 505, + 509, + 530, + 527, + 477, + 492, + 445, + 500, + 469, + 465, + 526, + 505, + 486, + 496, + 497, + 462, + 516, + 509, + 479, + 523, + 473, + 502, + 456, + 496, + 488, + 459, + 453, + 472, + 466, + 439, + 526, + 484, + 482, + 533, + 496, + 480, + 494, + 459, + 503, + 477, + 475, + 504, + 499, + 458, + 496, + 502, + 494, + 465, + 479, + 522, + 448, + 472, + 509, + 465, + 524, + 497, + 490, + 482, + 539, + 467, + 483, + 510, + 498, + 486, + 504, + 522, + 481, + 487, + 448, + 444, + 514, + 498, + 459, + 484, + 511, + 519, + 475, + 501, + 492, + 518, + 476, + 493, + 501, + 498, + 480, + 501, + 489, + 483, + 498, + 527, + 464, + 535, + 551, + 486, + 460, + 501, + 501, + 478, + 503, + 479, + 511, + 418, + 504, + 508, + 474, + 476, + 443, + 464, + 490, + 498, + 476, + 466, + 498, + 449, + 504, + 494, + 484, + 455, + 470, + 496, + 519, + 490, + 510, + 516, + 473, + 487, + 504, + 496, + 475, + 486, + 499, + 458, + 505, + 526, + 470, + 494, + 538, + 505, + 468, + 497, + 488, + 515, + 508, + 493, + 471, + 467, + 484, + 468, + 516, + 518, + 469, + 497, + 521, + 472, + 523, + 495, + 494, + 487, + 502, + 496, + 489, + 493, + 453, + 510, + 495, + 468, + 492, + 477, + 509, + 511, + 449, + 508, + 490, + 476, + 473, + 464, + 495, + 516, + 475, + 550, + 470, + 445, + 481, + 504, + 468, + 513, + 474, + 485, + 495, + 499, + 504, + 505, + 472, + 474, + 510, + 482, + 453, + 547, + 484, + 473, + 505, + 496, + 495, + 517, + 504, + 480, + 506, + 543, + 476, + 515, + 470, + 493, + 472, + 465, + 527, + 510, + 508, + 453, + 466, + 492, + 463, + 503, + 494, + 447, + 468, + 538, + 479, + 464, + 500, + 505, + 474, + 496, + 479, + 470, + 507, + 480, + 492, + 494, + 482, + 471, + 492, + 456, + 502, + 480, + 493, + 453, + 475, + 466, + 510, + 512, + 480, + 469, + 524, + 508, + 487, + 472, + 476, + 479, + 531, + 462, + 509, + 434, + 478, + 489, + 489, + 527, + 441, + 536, + 497, + 508, + 494, + 502, + 459, + 443, + 513, + 503, + 470, + 511, + 453, + 495, + 517, + 543, + 455, + 509, + 431, + 523, + 476, + 538, + 488, + 481, + 487, + 505, + 527, + 458, + 494, + 511, + 424, + 511, + 506, + 482, + 467, + 481, + 538, + 457, + 487, + 520, + 479, + 508, + 491, + 472, + 499, + 518, + 460, + 486, + 494, + 505, + 505, + 489, + 520, + 473, + 487, + 488, + 448, + 499, + 498, + 493, + 491, + 450, + 507, + 521, + 500, + 442, + 476, + 497, + 483, + 467, + 490, + 538, + 479, + 448, + 509, + 469, + 533, + 473, + 447, + 501, + 503, + 484, + 475, + 504, + 455, + 519, + 466, + 497, + 470, + 497, + 490, + 501, + 510, + 479, + 504, + 487, + 498, + 490, + 474, + 464, + 484, + 470, + 536, + 486, + 469, + 496, + 511, + 490, + 492, + 539, + 496, + 519, + 455, + 476, + 463, + 432, + 491, + 523, + 495, + 463, + 516, + 514, + 518, + 500, + 485, + 501, + 515, + 492, + 508, + 453, + 507, + 449, + 473, + 492, + 506, + 494, + 529, + 502, + 499, + 525, + 488, + 495, + 486, + 527, + 492, + 501, + 512, + 456, + 462, + 468, + 492, + 480, + 499, + 460, + 448, + 500, + 491, + 485, + 470, + 455, + 443, + 481, + 480, + 488, + 485, + 476, + 463, + 514, + 483, + 515, + 464, + 460, + 465, + 476, + 447, + 475, + 521, + 500, + 458, + 495, + 468, + 495, + 487, + 517, + 473, + 479, + 501, + 498, + 491, + 519, + 496, + 508, + 478, + 496, + 516, + 489, + 486, + 491, + 487, + 494, + 461, + 487, + 496, + 488, + 510, + 479, + 479, + 499, + 476, + 482, + 500, + 522, + 488, + 476, + 493, + 504, + 517, + 509, + 505, + 472, + 511, + 452, + 477, + 469, + 493, + 542, + 472, + 498, + 475, + 509, + 499, + 486, + 484, + 516, + 524, + 477, + 490, + 550, + 511, + 452, + 493, + 532, + 489, + 463, + 496, + 467, + 439, + 486, + 497, + 476, + 493, + 498, + 521, + 475, + 521, + 423, + 469, + 515, + 484, + 492, + 496, + 459, + 444, + 473, + 463, + 494, + 453, + 469, + 510, + 473, + 457, + 492, + 533, + 548, + 485, + 466, + 496, + 485, + 466, + 510, + 454, + 475, + 512, + 491, + 510, + 495, + 495, + 490, + 488, + 417, + 479, + 471, + 502, + 479, + 488, + 501, + 497, + 493, + 468, + 502, + 479, + 497, + 512, + 455, + 457, + 504, + 474, + 495, + 517, + 476, + 492, + 475, + 490, + 502, + 467, + 493, + 540, + 512, + 470, + 490, + 517, + 515, + 463, + 480, + 500, + 495, + 438, + 468, + 488, + 478, + 504, + 468, + 499, + 468, + 514, + 484, + 512, + 519, + 490, + 476, + 483, + 513, + 483, + 491, + 505, + 493, + 493, + 493, + 465, + 497, + 477, + 497, + 468, + 448, + 496, + 483, + 480, + 482, + 495, + 483, + 506, + 518, + 486, + 501, + 482, + 442, + 489, + 516, + 487, + 458, + 496, + 527, + 467, + 454, + 485, + 489, + 496, + 469, + 468, + 523, + 470, + 480, + 477, + 471, + 506, + 480, + 522, + 499, + 479, + 483, + 537, + 466, + 485, + 490, + 534, + 468, + 500, + 514, + 494, + 489, + 480, + 511, + 509, + 488, + 544, + 492, + 478, + 536, + 447, + 465, + 490, + 457, + 501, + 501, + 533, + 498, + 486, + 456, + 491, + 487, + 477, + 494, + 501, + 492, + 472, + 459, + 491, + 490, + 461, + 499, + 485, + 489, + 476, + 477, + 503, + 476, + 483, + 470, + 496, + 450, + 467, + 472, + 500, + 488, + 482, + 471, + 490, + 494, + 482, + 486, + 551, + 502, + 510, + 496, + 495, + 524, + 521, + 504, + 483, + 488, + 487, + 489, + 506, + 525, + 469, + 495, + 473, + 462, + 495, + 472, + 507, + 480, + 507, + 517, + 509, + 503, + 504, + 478, + 541, + 471, + 517, + 520, + 516, + 506, + 515, + 479, + 496, + 473, + 469, + 459, + 488, + 487, + 463, + 493, + 492, + 484, + 487, + 462, + 475, + 493, + 541, + 492, + 480, + 492, + 522, + 522, + 463, + 490, + 485, + 540, + 482, + 500, + 520, + 468, + 529, + 478, + 522, + 527, + 465, + 430, + 501, + 485, + 474, + 517, + 509, + 491, + 484, + 468, + 505 + ] + } + ], + "default_value": 0 + }, + "null_count": 0, + "tot_col_size": 0, + "last_update_version": 417600050823954432, + "correlation": 0 + } + }, + "ext_stats": [ + { + "stats_name": "stats1", + "db": "test", + "cols": [ + 1, + 2 + ], + "type": 2, + "scalar_vals": -1, + "string_vals": "" + } + ], + "count": 1000000, + "modify_count": 0, + "partitions": null +} \ No newline at end of file diff --git a/planner/core/testdata/analyze_suite_in.json b/planner/core/testdata/analyze_suite_in.json index 1187de96f090e..8d2e32ddb7401 100644 --- a/planner/core/testdata/analyze_suite_in.json +++ b/planner/core/testdata/analyze_suite_in.json @@ -193,5 +193,14 @@ // Should choose `primary` index instead of index `b`. "explain select c, b from t where a = 7639902 order by b asc limit 6" ] + }, + { + "name": "TestLimitIndexEstimation", + "cases": [ + // Should choose idx_a instead of idx_b, because idx_b would scan 990001 rows. + "explain select * from t where a <= 10000 order by b limit 1", + // Should choose idx_b instead of idx_a, because idx_b would scan only 1 row. + "explain select * from t where a >= 999900 order by b limit 1" + ] } ] diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index 8d42349bb20fc..e7d6df463ef1c 100644 --- a/planner/core/testdata/analyze_suite_out.json +++ b/planner/core/testdata/analyze_suite_out.json @@ -378,5 +378,31 @@ ] } ] + }, + { + "Name": "TestLimitIndexEstimation", + "Cases": [ + { + "SQL": "explain select * from t where a <= 10000 order by b limit 1", + "Plan": [ + "TopN_9 1.00 root test.t.b, offset:0, count:1", + "└─IndexLookUp_21 1.00 root ", + " ├─IndexRangeScan_18(Build) 10000.00 cop[tikv] table:t, index:idx_a(a) range:[-inf,10000], keep order:false", + " └─TopN_20(Probe) 1.00 cop[tikv] test.t.b, offset:0, count:1", + " └─TableRowIDScan_19 10000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain select * from t where a >= 999900 order by b limit 1", + "Plan": [ + "Limit_12 1.00 root offset:0, count:1", + "└─Projection_32 1.00 root test.t.a, test.t.b", + " └─IndexLookUp_31 1.00 root ", + " ├─IndexFullScan_28(Build) 1.00 cop[tikv] table:t, index:idx_b(b) keep order:true", + " └─Selection_30(Probe) 1.00 cop[tikv] ge(test.t.a, 999900)", + " └─TableRowIDScan_29 1.00 cop[tikv] table:t keep order:false" + ] + } + ] } ] diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 62930182da16d..2417855ee0660 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -19,6 +19,13 @@ "explain select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);" ] }, + { + "name": "TestIndexJoinInnerIndexNDV", + "cases": [ + // t2 should use idx2 instead of idx1, since idx2 has larger NDV. + "explain select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c" + ] + }, { "name": "TestSimplifyOuterJoinWithCast", "cases": [ diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index fcbd1959212e1..245f8f991f385 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -62,6 +62,22 @@ } ] }, + { + "Name": "TestIndexJoinInnerIndexNDV", + "Cases": [ + { + "SQL": "explain select /*+ inl_join(t2) */ * from t1, t2 where t1.a = t2.a and t1.b = t2.b and t1.c = t2.c", + "Plan": [ + "IndexJoin_10 3.00 root inner join, inner:IndexLookUp_9, outer key:test.t1.c, inner key:test.t2.c, other cond:eq(test.t1.a, test.t2.a), eq(test.t1.b, test.t2.b)", + "├─TableReader_20(Build) 3.00 root data:TableFullScan_19", + "│ └─TableFullScan_19 3.00 cop[tikv] table:t1 keep order:false", + "└─IndexLookUp_9(Probe) 1.00 root ", + " ├─IndexRangeScan_7(Build) 1.00 cop[tikv] table:t2, index:idx2(c) range: decided by [eq(test.t2.c, test.t1.c)], keep order:false", + " └─TableRowIDScan_8(Probe) 1.00 cop[tikv] table:t2 keep order:false" + ] + } + ] + }, { "Name": "TestSimplifyOuterJoinWithCast", "Cases": [ diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index acf3a9c160267..4fba9a92251d2 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -879,7 +879,7 @@ "Cases": [ { "SQL": "select a from t where c is not null", - "Best": "IndexReader(Index(t.c_d_e)[[-inf,+inf]])->Projection" + "Best": "IndexReader(Index(t.f)[[NULL,+inf]])" }, { "SQL": "select a from t where c >= 4", @@ -1225,17 +1225,17 @@ }, { "SQL": "select /*+ HASH_AGG() */ t1.a from t t1 where t1.a < any(select t2.b from t t2)", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([if(isnull(test.t.a), , 1)]))->TableReader(Table(t)->HashAgg)->HashAgg->Sel([ne(Column#27, 0)])}", + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->TableReader(Table(t)->Sel([1])->HashAgg)->HashAgg->Sel([ne(Column#27, 0) 1])}", "Warning": "" }, { "SQL": "select /*+ hash_agg() */ t1.a from t t1 where t1.a != any(select t2.b from t t2)", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([if(isnull(test.t.a), , 1)]))->TableReader(Table(t))->Projection->HashAgg->Sel([ne(Column#28, 0)])}", + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->TableReader(Table(t)->Sel([1]))->HashAgg->Sel([ne(Column#28, 0) 1])}", "Warning": "" }, { "SQL": "select /*+ hash_agg() */ t1.a from t t1 where t1.a = all(select t2.b from t t2)", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->TableReader(Table(t))->Projection->HashAgg}", + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->TableReader(Table(t))->HashAgg->Sel([or(and(le(Column#26, 1), if(ne(Column#27, 0), , 1)), or(eq(Column#28, 0), 0))])}", "Warning": "" }, { diff --git a/planner/core/testdata/plan_suite_unexported_in.json b/planner/core/testdata/plan_suite_unexported_in.json index b71d9f9960da7..41aef7bec4ccc 100644 --- a/planner/core/testdata/plan_suite_unexported_in.json +++ b/planner/core/testdata/plan_suite_unexported_in.json @@ -402,6 +402,14 @@ "select count(1) from (select count(b) as cnt from t group by c) t1" ] }, + { + "name": "TestSortByItemsPruning", + "cases": [ + "select * from t where a > 1 order by a asc, a asc limit 10", + "select * from t where a > 1 order by a asc, b asc, a asc, c asc limit 10", + "select * from t where a > 1 order by pow(a, 2) asc, b asc, pow(a, 2) asc, c asc limit 10" + ] + }, { "name": "TestDeriveNotNullConds", "cases": [ diff --git a/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json index ade08e163fe7e..699b0d3214326 100644 --- a/planner/core/testdata/plan_suite_unexported_out.json +++ b/planner/core/testdata/plan_suite_unexported_out.json @@ -739,6 +739,24 @@ } ] }, + { + "Name": "TestSortByItemsPruning", + "Cases": [ + [ + "test.t.a" + ], + [ + "test.t.a", + "test.t.b", + "test.t.c" + ], + [ + "pow(cast(test.t.a, double BINARY), 2)", + "test.t.b", + "test.t.c" + ] + ] + }, { "Name": "TestDeriveNotNullConds", "Cases": [ diff --git a/server/conn.go b/server/conn.go index fe6d8abaaa52f..0f2d50e7b5f3e 100644 --- a/server/conn.go +++ b/server/conn.go @@ -173,6 +173,12 @@ type clientConn struct { status int32 // dispatching/reading/shutdown/waitshutdown lastCode uint16 // last error code collation uint8 // collation used by client, may be different from the collation used by database. + + // mu is used for cancelling the execution of current transaction. + mu struct { + sync.RWMutex + cancelFunc context.CancelFunc + } } func (cc *clientConn) String() string { @@ -910,6 +916,13 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { atomic.StoreUint32(&cc.ctx.GetSessionVars().Killed, 0) }() span := opentracing.StartSpan("server.dispatch") + ctx = opentracing.ContextWithSpan(ctx, span) + + var cancelFunc context.CancelFunc + ctx, cancelFunc = context.WithCancel(ctx) + cc.mu.Lock() + cc.mu.cancelFunc = cancelFunc + cc.mu.Unlock() t := time.Now() cc.lastPacket = data @@ -1387,6 +1400,17 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { var pointPlans []plannercore.Plan if len(stmts) > 1 { + + // The client gets to choose if it allows multi-statements, and + // probably defaults OFF. This helps prevent against SQL injection attacks + // by early terminating the first statement, and then running an entirely + // new statement. + + capabilities := cc.ctx.GetSessionVars().ClientCapability + if capabilities&mysql.ClientMultiStatements < 1 { + return errMultiStatementDisabled + } + // Only pre-build point plans for multi-statement query pointPlans, err = cc.prefetchPointPlanKeys(ctx, stmts) if err != nil { @@ -1427,7 +1451,7 @@ func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.Stm } vars := cc.ctx.GetSessionVars() if vars.TxnCtx.IsPessimistic { - if vars.IsReadConsistencyTxn() { + if vars.IsIsolation(ast.ReadCommitted) { // TODO: to support READ-COMMITTED, we need to avoid getting new TS for each statement in the query. return nil, nil } diff --git a/server/conn_test.go b/server/conn_test.go index dd4685e7c814a..8f94bda3c746d 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -680,6 +680,11 @@ func (ts *ConnTestSuite) TestPrefetchPointKeys(c *C) { tk.MustExec("insert prefetch values (1, 1, 1), (2, 2, 2), (3, 3, 3)") tk.MustExec("begin optimistic") tk.MustExec("update prefetch set c = c + 1 where a = 2 and b = 2") + + // enable multi-statement + capabilities := cc.ctx.GetSessionVars().ClientCapability + capabilities ^= mysql.ClientMultiStatements + cc.ctx.SetClientCapability(capabilities) query := "update prefetch set c = c + 1 where a = 1 and b = 1;" + "update prefetch set c = c + 1 where a = 2 and b = 2;" + "update prefetch set c = c + 1 where a = 3 and b = 3;" diff --git a/server/http_handler.go b/server/http_handler.go old mode 100755 new mode 100644 diff --git a/server/server.go b/server/server.go index 8954eca113a77..88f5a75882811 100644 --- a/server/server.go +++ b/server/server.go @@ -97,6 +97,7 @@ var ( errAccessDenied = terror.ClassServer.New(errno.ErrAccessDenied, errno.MySQLErrName[errno.ErrAccessDenied]) errConCount = terror.ClassServer.New(errno.ErrConCount, errno.MySQLErrName[errno.ErrConCount]) errSecureTransportRequired = terror.ClassServer.New(errno.ErrSecureTransportRequired, errno.MySQLErrName[errno.ErrSecureTransportRequired]) + errMultiStatementDisabled = terror.ClassServer.New(errno.ErrUnknown, "client has multi-statement capability disabled") // MySQL returns a parse error ) // DefaultCapability is the capability of the server when it is created using the default configuration. @@ -539,6 +540,12 @@ func (s *Server) getTLSConfig() *tls.Config { func killConn(conn *clientConn) { sessVars := conn.ctx.GetSessionVars() atomic.StoreUint32(&sessVars.Killed, 1) + conn.mu.RLock() + cancelFunc := conn.mu.cancelFunc + conn.mu.RUnlock() + if cancelFunc != nil { + cancelFunc() + } } // KillAllConnections kills all connections when server is not gracefully shutdown. diff --git a/server/server_test.go b/server/server_test.go index b400b54be69f5..feb14cb4ccffd 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -1212,8 +1212,22 @@ func (cli *testServerClient) runTestStatusAPI(c *C) { c.Assert(data.GitHash, Equals, versioninfo.TiDBGitHash) } +// The golang sql driver (and most drivers) should have multi-statement +// disabled by default for security reasons. Lets ensure that the behavior +// is correct. + +func (cli *testServerClient) runFailedTestMultiStatements(c *C) { + cli.runTestsOnNewDB(c, nil, "FailedMultiStatements", func(dbt *DBTest) { + _, err := dbt.db.Exec("SELECT 1; SELECT 1; SELECT 2; SELECT 3;") + c.Assert(err.Error(), Equals, "Error 1105: client has multi-statement capability disabled") + }) +} + func (cli *testServerClient) runTestMultiStatements(c *C) { - cli.runTestsOnNewDB(c, nil, "MultiStatements", func(dbt *DBTest) { + + cli.runTestsOnNewDB(c, func(config *mysql.Config) { + config.Params = map[string]string{"multiStatements": "true"} + }, "MultiStatements", func(dbt *DBTest) { // Create Table dbt.mustExec("CREATE TABLE `test` (`id` int(11) NOT NULL, `value` int(11) NOT NULL) ") diff --git a/server/tidb_test.go b/server/tidb_test.go index 2cf4f07b7752a..cdabeb1c2d27c 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -311,6 +311,7 @@ func newTLSHttpClient(c *C, caFile, certFile, keyFile string) *http.Client { func (ts *tidbTestSuite) TestMultiStatements(c *C) { c.Parallel() + ts.runFailedTestMultiStatements(c) ts.runTestMultiStatements(c) } diff --git a/session/bootstrap.go b/session/bootstrap.go index 1551a84336cb0..88bb3f62c56e3 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -45,23 +45,23 @@ import ( const ( // CreateUserTable is the SQL statement creates User table in system db. - CreateUserTable = `CREATE TABLE if not exists mysql.user ( - Host CHAR(64), - User CHAR(32), + CreateUserTable = `CREATE TABLE IF NOT EXISTS mysql.user ( + Host CHAR(64), + User CHAR(32), authentication_string TEXT, - Select_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Insert_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Update_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Delete_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Create_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Drop_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Select_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Insert_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Update_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Delete_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Create_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Drop_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Process_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Grant_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Grant_priv ENUM('N','Y') NOT NULL DEFAULT 'N', References_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Alter_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Alter_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Show_db_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Super_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Create_tmp_table_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Super_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Create_tmp_table_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Lock_tables_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Execute_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Create_view_priv ENUM('N','Y') NOT NULL DEFAULT 'N', @@ -82,210 +82,210 @@ const ( Create_Tablespace_Priv ENUM('N','Y') NOT NULL DEFAULT 'N', PRIMARY KEY (Host, User));` // CreateGlobalPrivTable is the SQL statement creates Global scope privilege table in system db. - CreateGlobalPrivTable = "CREATE TABLE if not exists mysql.global_priv (" + - "Host char(60) NOT NULL DEFAULT ''," + - "User char(80) NOT NULL DEFAULT ''," + - "Priv longtext NOT NULL DEFAULT ''," + + CreateGlobalPrivTable = "CREATE TABLE IF NOT EXISTS mysql.global_priv (" + + "Host CHAR(60) NOT NULL DEFAULT ''," + + "User CHAR(80) NOT NULL DEFAULT ''," + + "Priv LONGTEXT NOT NULL DEFAULT ''," + "PRIMARY KEY (Host, User)" + ")" // CreateDBPrivTable is the SQL statement creates DB scope privilege table in system db. - CreateDBPrivTable = `CREATE TABLE if not exists mysql.db ( - Host CHAR(60), - DB CHAR(64), - User CHAR(32), - Select_priv ENUM('N','Y') Not Null DEFAULT 'N', - Insert_priv ENUM('N','Y') Not Null DEFAULT 'N', - Update_priv ENUM('N','Y') Not Null DEFAULT 'N', - Delete_priv ENUM('N','Y') Not Null DEFAULT 'N', - Create_priv ENUM('N','Y') Not Null DEFAULT 'N', - Drop_priv ENUM('N','Y') Not Null DEFAULT 'N', - Grant_priv ENUM('N','Y') Not Null DEFAULT 'N', - References_priv ENUM('N','Y') Not Null DEFAULT 'N', - Index_priv ENUM('N','Y') Not Null DEFAULT 'N', - Alter_priv ENUM('N','Y') Not Null DEFAULT 'N', + CreateDBPrivTable = `CREATE TABLE IF NOT EXISTS mysql.db ( + Host CHAR(60), + DB CHAR(64), + User CHAR(32), + Select_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Insert_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Update_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Delete_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Create_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Drop_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Grant_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + References_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Index_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Alter_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Create_tmp_table_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Lock_tables_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Create_view_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Show_view_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Create_routine_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Alter_routine_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Execute_priv ENUM('N','Y') Not Null DEFAULT 'N', - Event_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - Trigger_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Lock_tables_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Create_view_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Show_view_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Create_routine_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Alter_routine_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Execute_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Event_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Trigger_priv ENUM('N','Y') NOT NULL DEFAULT 'N', PRIMARY KEY (Host, DB, User));` // CreateTablePrivTable is the SQL statement creates table scope privilege table in system db. - CreateTablePrivTable = `CREATE TABLE if not exists mysql.tables_priv ( + CreateTablePrivTable = `CREATE TABLE IF NOT EXISTS mysql.tables_priv ( Host CHAR(60), - DB CHAR(64), + DB CHAR(64), User CHAR(32), Table_name CHAR(64), Grantor CHAR(77), - Timestamp Timestamp DEFAULT CURRENT_TIMESTAMP, + Timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, Table_priv SET('Select','Insert','Update','Delete','Create','Drop','Grant','Index','Alter','Create View','Show View','Trigger','References'), Column_priv SET('Select','Insert','Update'), PRIMARY KEY (Host, DB, User, Table_name));` // CreateColumnPrivTable is the SQL statement creates column scope privilege table in system db. - CreateColumnPrivTable = `CREATE TABLE if not exists mysql.columns_priv( + CreateColumnPrivTable = `CREATE TABLE IF NOT EXISTS mysql.columns_priv( Host CHAR(60), - DB CHAR(64), + DB CHAR(64), User CHAR(32), Table_name CHAR(64), Column_name CHAR(64), - Timestamp Timestamp DEFAULT CURRENT_TIMESTAMP, + Timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, Column_priv SET('Select','Insert','Update'), PRIMARY KEY (Host, DB, User, Table_name, Column_name));` // CreateGlobalVariablesTable is the SQL statement creates global variable table in system db. // TODO: MySQL puts GLOBAL_VARIABLES table in INFORMATION_SCHEMA db. // INFORMATION_SCHEMA is a virtual db in TiDB. So we put this table in system db. // Maybe we will put it back to INFORMATION_SCHEMA. - CreateGlobalVariablesTable = `CREATE TABLE if not exists mysql.GLOBAL_VARIABLES( - VARIABLE_NAME VARCHAR(64) Not Null PRIMARY KEY, - VARIABLE_VALUE VARCHAR(1024) DEFAULT Null);` + CreateGlobalVariablesTable = `CREATE TABLE IF NOT EXISTS mysql.GLOBAL_VARIABLES( + VARIABLE_NAME VARCHAR(64) NOT NULL PRIMARY KEY, + VARIABLE_VALUE VARCHAR(1024) DEFAULT NULL);` // CreateTiDBTable is the SQL statement creates a table in system db. // This table is a key-value struct contains some information used by TiDB. // Currently we only put bootstrapped in it which indicates if the system is already bootstrapped. - CreateTiDBTable = `CREATE TABLE if not exists mysql.tidb( - VARIABLE_NAME VARCHAR(64) Not Null PRIMARY KEY, - VARIABLE_VALUE VARCHAR(1024) DEFAULT Null, - COMMENT VARCHAR(1024));` + CreateTiDBTable = `CREATE TABLE IF NOT EXISTS mysql.tidb( + VARIABLE_NAME VARCHAR(64) NOT NULL PRIMARY KEY, + VARIABLE_VALUE VARCHAR(1024) DEFAULT NULL, + COMMENT VARCHAR(1024));` // CreateHelpTopic is the SQL statement creates help_topic table in system db. // See: https://dev.mysql.com/doc/refman/5.5/en/system-database.html#system-database-help-tables - CreateHelpTopic = `CREATE TABLE if not exists mysql.help_topic ( - help_topic_id int(10) unsigned NOT NULL, - name char(64) NOT NULL, - help_category_id smallint(5) unsigned NOT NULL, - description text NOT NULL, - example text NOT NULL, - url text NOT NULL, + CreateHelpTopic = `CREATE TABLE IF NOT EXISTS mysql.help_topic ( + help_topic_id INT(10) UNSIGNED NOT NULL, + name CHAR(64) NOT NULL, + help_category_id SMALLINT(5) UNSIGNED NOT NULL, + description TEXT NOT NULL, + example TEXT NOT NULL, + url TEXT NOT NULL, PRIMARY KEY (help_topic_id), UNIQUE KEY name (name) ) ENGINE=InnoDB DEFAULT CHARSET=utf8 STATS_PERSISTENT=0 COMMENT='help topics';` // CreateStatsMetaTable stores the meta of table statistics. - CreateStatsMetaTable = `CREATE TABLE if not exists mysql.stats_meta ( - version bigint(64) unsigned NOT NULL, - table_id bigint(64) NOT NULL, - modify_count bigint(64) NOT NULL DEFAULT 0, - count bigint(64) unsigned NOT NULL DEFAULT 0, - index idx_ver(version), - unique index tbl(table_id) + CreateStatsMetaTable = `CREATE TABLE IF NOT EXISTS mysql.stats_meta ( + version BIGINT(64) UNSIGNED NOT NULL, + table_id BIGINT(64) NOT NULL, + modify_count BIGINT(64) NOT NULL DEFAULT 0, + count BIGINT(64) UNSIGNED NOT NULL DEFAULT 0, + INDEX idx_ver(version), + UNIQUE INDEX tbl(table_id) );` // CreateStatsColsTable stores the statistics of table columns. - CreateStatsColsTable = `CREATE TABLE if not exists mysql.stats_histograms ( - table_id bigint(64) NOT NULL, - is_index tinyint(2) NOT NULL, - hist_id bigint(64) NOT NULL, - distinct_count bigint(64) NOT NULL, - null_count bigint(64) NOT NULL DEFAULT 0, - tot_col_size bigint(64) NOT NULL DEFAULT 0, - modify_count bigint(64) NOT NULL DEFAULT 0, - version bigint(64) unsigned NOT NULL DEFAULT 0, - cm_sketch blob, - stats_ver bigint(64) NOT NULL DEFAULT 0, - flag bigint(64) NOT NULL DEFAULT 0, - correlation double NOT NULL DEFAULT 0, - last_analyze_pos blob DEFAULT NULL, - unique index tbl(table_id, is_index, hist_id) + CreateStatsColsTable = `CREATE TABLE IF NOT EXISTS mysql.stats_histograms ( + table_id BIGINT(64) NOT NULL, + is_index TINYINT(2) NOT NULL, + hist_id BIGINT(64) NOT NULL, + distinct_count BIGINT(64) NOT NULL, + null_count BIGINT(64) NOT NULL DEFAULT 0, + tot_col_size BIGINT(64) NOT NULL DEFAULT 0, + modify_count BIGINT(64) NOT NULL DEFAULT 0, + version BIGINT(64) UNSIGNED NOT NULL DEFAULT 0, + cm_sketch BLOB, + stats_ver BIGINT(64) NOT NULL DEFAULT 0, + flag BIGINT(64) NOT NULL DEFAULT 0, + correlation DOUBLE NOT NULL DEFAULT 0, + last_analyze_pos BLOB DEFAULT NULL, + UNIQUE INDEX tbl(table_id, is_index, hist_id) );` // CreateStatsBucketsTable stores the histogram info for every table columns. - CreateStatsBucketsTable = `CREATE TABLE if not exists mysql.stats_buckets ( - table_id bigint(64) NOT NULL, - is_index tinyint(2) NOT NULL, - hist_id bigint(64) NOT NULL, - bucket_id bigint(64) NOT NULL, - count bigint(64) NOT NULL, - repeats bigint(64) NOT NULL, - upper_bound blob NOT NULL, - lower_bound blob , - unique index tbl(table_id, is_index, hist_id, bucket_id) + CreateStatsBucketsTable = `CREATE TABLE IF NOT EXISTS mysql.stats_buckets ( + table_id BIGINT(64) NOT NULL, + is_index TINYINT(2) NOT NULL, + hist_id BIGINT(64) NOT NULL, + bucket_id BIGINT(64) NOT NULL, + count BIGINT(64) NOT NULL, + repeats BIGINT(64) NOT NULL, + upper_bound BLOB NOT NULL, + lower_bound BLOB , + UNIQUE INDEX tbl(table_id, is_index, hist_id, bucket_id) );` // CreateGCDeleteRangeTable stores schemas which can be deleted by DeleteRange. CreateGCDeleteRangeTable = `CREATE TABLE IF NOT EXISTS mysql.gc_delete_range ( - job_id BIGINT NOT NULL COMMENT "the DDL job ID", - element_id BIGINT NOT NULL COMMENT "the schema element ID", - start_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", - end_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", - ts BIGINT NOT NULL COMMENT "timestamp in uint64", + job_id BIGINT NOT NULL COMMENT "the DDL job ID", + element_id BIGINT NOT NULL COMMENT "the schema element ID", + start_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", + end_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", + ts BIGINT NOT NULL COMMENT "timestamp in uint64", UNIQUE KEY delete_range_index (job_id, element_id) );` // CreateGCDeleteRangeDoneTable stores schemas which are already deleted by DeleteRange. CreateGCDeleteRangeDoneTable = `CREATE TABLE IF NOT EXISTS mysql.gc_delete_range_done ( - job_id BIGINT NOT NULL COMMENT "the DDL job ID", - element_id BIGINT NOT NULL COMMENT "the schema element ID", - start_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", - end_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", - ts BIGINT NOT NULL COMMENT "timestamp in uint64", + job_id BIGINT NOT NULL COMMENT "the DDL job ID", + element_id BIGINT NOT NULL COMMENT "the schema element ID", + start_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", + end_key VARCHAR(255) NOT NULL COMMENT "encoded in hex", + ts BIGINT NOT NULL COMMENT "timestamp in uint64", UNIQUE KEY delete_range_done_index (job_id, element_id) );` // CreateStatsFeedbackTable stores the feedback info which is used to update stats. CreateStatsFeedbackTable = `CREATE TABLE IF NOT EXISTS mysql.stats_feedback ( - table_id bigint(64) NOT NULL, - is_index tinyint(2) NOT NULL, - hist_id bigint(64) NOT NULL, - feedback blob NOT NULL, - index hist(table_id, is_index, hist_id) + table_id BIGINT(64) NOT NULL, + is_index TINYINT(2) NOT NULL, + hist_id BIGINT(64) NOT NULL, + feedback BLOB NOT NULL, + INDEX hist(table_id, is_index, hist_id) );` // CreateBindInfoTable stores the sql bind info which is used to update globalBindCache. CreateBindInfoTable = `CREATE TABLE IF NOT EXISTS mysql.bind_info ( - original_sql text NOT NULL , - bind_sql text NOT NULL , - default_db text NOT NULL, - status text NOT NULL, - create_time timestamp(3) NOT NULL, - update_time timestamp(3) NOT NULL, - charset text NOT NULL, - collation text NOT NULL, - source varchar(10) NOT NULL default 'unknown', + original_sql TEXT NOT NULL , + bind_sql TEXT NOT NULL , + default_db TEXT NOT NULL, + status TEXT NOT NULL, + create_time TIMESTAMP(3) NOT NULL, + update_time TIMESTAMP(3) NOT NULL, + charset TEXT NOT NULL, + collation TEXT NOT NULL, + source VARCHAR(10) NOT NULL DEFAULT 'unknown', INDEX sql_index(original_sql(1024),default_db(1024)) COMMENT "accelerate the speed when add global binding query", INDEX time_index(update_time) COMMENT "accelerate the speed when querying with last update time" ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;` // CreateRoleEdgesTable stores the role and user relationship information. CreateRoleEdgesTable = `CREATE TABLE IF NOT EXISTS mysql.role_edges ( - FROM_HOST char(60) COLLATE utf8_bin NOT NULL DEFAULT '', - FROM_USER char(32) COLLATE utf8_bin NOT NULL DEFAULT '', - TO_HOST char(60) COLLATE utf8_bin NOT NULL DEFAULT '', - TO_USER char(32) COLLATE utf8_bin NOT NULL DEFAULT '', - WITH_ADMIN_OPTION enum('N','Y') CHARACTER SET utf8 COLLATE utf8_general_ci NOT NULL DEFAULT 'N', + FROM_HOST CHAR(60) COLLATE utf8_bin NOT NULL DEFAULT '', + FROM_USER CHAR(32) COLLATE utf8_bin NOT NULL DEFAULT '', + TO_HOST CHAR(60) COLLATE utf8_bin NOT NULL DEFAULT '', + TO_USER CHAR(32) COLLATE utf8_bin NOT NULL DEFAULT '', + WITH_ADMIN_OPTION ENUM('N','Y') CHARACTER SET utf8 COLLATE utf8_general_ci NOT NULL DEFAULT 'N', PRIMARY KEY (FROM_HOST,FROM_USER,TO_HOST,TO_USER) );` // CreateDefaultRolesTable stores the active roles for a user. CreateDefaultRolesTable = `CREATE TABLE IF NOT EXISTS mysql.default_roles ( - HOST char(60) COLLATE utf8_bin NOT NULL DEFAULT '', - USER char(32) COLLATE utf8_bin NOT NULL DEFAULT '', - DEFAULT_ROLE_HOST char(60) COLLATE utf8_bin NOT NULL DEFAULT '%', - DEFAULT_ROLE_USER char(32) COLLATE utf8_bin NOT NULL DEFAULT '', + HOST CHAR(60) COLLATE utf8_bin NOT NULL DEFAULT '', + USER CHAR(32) COLLATE utf8_bin NOT NULL DEFAULT '', + DEFAULT_ROLE_HOST CHAR(60) COLLATE utf8_bin NOT NULL DEFAULT '%', + DEFAULT_ROLE_USER CHAR(32) COLLATE utf8_bin NOT NULL DEFAULT '', PRIMARY KEY (HOST,USER,DEFAULT_ROLE_HOST,DEFAULT_ROLE_USER) )` // CreateStatsTopNTable stores topn data of a cmsketch with top n. - CreateStatsTopNTable = `CREATE TABLE if not exists mysql.stats_top_n ( - table_id bigint(64) NOT NULL, - is_index tinyint(2) NOT NULL, - hist_id bigint(64) NOT NULL, - value longblob, - count bigint(64) UNSIGNED NOT NULL, - index tbl(table_id, is_index, hist_id) + CreateStatsTopNTable = `CREATE TABLE IF NOT EXISTS mysql.stats_top_n ( + table_id BIGINT(64) NOT NULL, + is_index TINYINT(2) NOT NULL, + hist_id BIGINT(64) NOT NULL, + value LONGBLOB, + count BIGINT(64) UNSIGNED NOT NULL, + INDEX tbl(table_id, is_index, hist_id) );` // CreateExprPushdownBlacklist stores the expressions which are not allowed to be pushed down. CreateExprPushdownBlacklist = `CREATE TABLE IF NOT EXISTS mysql.expr_pushdown_blacklist ( - name char(100) NOT NULL, - store_type char(100) NOT NULL DEFAULT 'tikv,tiflash,tidb', - reason varchar(200) + name CHAR(100) NOT NULL, + store_type CHAR(100) NOT NULL DEFAULT 'tikv,tiflash,tidb', + reason VARCHAR(200) );` // CreateOptRuleBlacklist stores the list of disabled optimizing operations. CreateOptRuleBlacklist = `CREATE TABLE IF NOT EXISTS mysql.opt_rule_blacklist ( - name char(100) NOT NULL + name CHAR(100) NOT NULL );` // CreateStatsExtended stores the registered extended statistics. @@ -598,7 +598,7 @@ func upgradeToVer3(s Session, ver int64) { return } // Version 3 fix tx_read_only variable value. - sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %s.%s set variable_value = '0' where variable_name = 'tx_read_only';", + sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %s.%s SET variable_value = '0' WHERE variable_name = 'tx_read_only';", mysql.SystemDB, mysql.GlobalVariablesTable) mustExecute(s, sql) } @@ -624,7 +624,7 @@ func upgradeToVer6(s Session, ver int64) { if ver >= version6 { return } - doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Super_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Show_db_priv`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Super_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Show_db_priv`", infoschema.ErrColumnExists) // For reasons of compatibility, set the non-exists privilege column value to 'Y', as TiDB doesn't check them in older versions. mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Super_priv='Y'") } @@ -633,7 +633,7 @@ func upgradeToVer7(s Session, ver int64) { if ver >= version7 { return } - doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Process_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Drop_priv`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Process_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Drop_priv`", infoschema.ErrColumnExists) // For reasons of compatibility, set the non-exists privilege column value to 'Y', as TiDB doesn't check them in older versions. mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Process_priv='Y'") } @@ -643,7 +643,7 @@ func upgradeToVer8(s Session, ver int64) { return } // This is a dummy upgrade, it checks whether upgradeToVer7 success, if not, do it again. - if _, err := s.Execute(context.Background(), "SELECT HIGH_PRIORITY `Process_priv` from mysql.user limit 0"); err == nil { + if _, err := s.Execute(context.Background(), "SELECT HIGH_PRIORITY `Process_priv` FROM mysql.user LIMIT 0"); err == nil { return } upgradeToVer7(s, ver) @@ -653,7 +653,7 @@ func upgradeToVer9(s Session, ver int64) { if ver >= version9 { return } - doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Trigger_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_user_priv`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Trigger_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_user_priv`", infoschema.ErrColumnExists) // For reasons of compatibility, set the non-exists privilege column value to 'Y', as TiDB doesn't check them in older versions. mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Trigger_priv='Y'") } @@ -676,7 +676,7 @@ func upgradeToVer10(s Session, ver int64) { } doReentrantDDL(s, "ALTER TABLE mysql.stats_buckets CHANGE COLUMN `value` `upper_bound` BLOB NOT NULL", infoschema.ErrColumnNotExists, infoschema.ErrColumnExists) doReentrantDDL(s, "ALTER TABLE mysql.stats_buckets ADD COLUMN `lower_bound` BLOB", infoschema.ErrColumnExists) - doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `null_count` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `null_count` BIGINT(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms DROP COLUMN distinct_ratio", ddl.ErrCantDropFieldOrKey) doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms DROP COLUMN use_count_to_estimate", ddl.ErrCantDropFieldOrKey) } @@ -685,7 +685,7 @@ func upgradeToVer11(s Session, ver int64) { if ver >= version11 { return } - _, err := s.Execute(context.Background(), "ALTER TABLE mysql.user ADD COLUMN `References_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Grant_priv`") + _, err := s.Execute(context.Background(), "ALTER TABLE mysql.user ADD COLUMN `References_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Grant_priv`") if err != nil { if terror.ErrorEqual(err, infoschema.ErrColumnExists) { return @@ -723,7 +723,7 @@ func upgradeToVer12(s Session, ver int64) { var newPass string newPass, err = oldPasswordUpgrade(pass) terror.MustNil(err) - updateSQL := fmt.Sprintf(`UPDATE HIGH_PRIORITY mysql.user set password = "%s" where user="%s" and host="%s"`, newPass, user, host) + updateSQL := fmt.Sprintf(`UPDATE HIGH_PRIORITY mysql.user SET password = "%s" WHERE user="%s" AND host="%s"`, newPass, user, host) sqls = append(sqls, updateSQL) } err = r.Next(ctx, req) @@ -746,13 +746,13 @@ func upgradeToVer13(s Session, ver int64) { return } sqls := []string{ - "ALTER TABLE mysql.user ADD COLUMN `Create_tmp_table_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Super_priv`", - "ALTER TABLE mysql.user ADD COLUMN `Lock_tables_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_tmp_table_priv`", - "ALTER TABLE mysql.user ADD COLUMN `Create_view_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Execute_priv`", - "ALTER TABLE mysql.user ADD COLUMN `Show_view_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_view_priv`", - "ALTER TABLE mysql.user ADD COLUMN `Create_routine_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Show_view_priv`", - "ALTER TABLE mysql.user ADD COLUMN `Alter_routine_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_routine_priv`", - "ALTER TABLE mysql.user ADD COLUMN `Event_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_user_priv`", + "ALTER TABLE mysql.user ADD COLUMN `Create_tmp_table_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Super_priv`", + "ALTER TABLE mysql.user ADD COLUMN `Lock_tables_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_tmp_table_priv`", + "ALTER TABLE mysql.user ADD COLUMN `Create_view_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Execute_priv`", + "ALTER TABLE mysql.user ADD COLUMN `Show_view_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_view_priv`", + "ALTER TABLE mysql.user ADD COLUMN `Create_routine_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Show_view_priv`", + "ALTER TABLE mysql.user ADD COLUMN `Alter_routine_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_routine_priv`", + "ALTER TABLE mysql.user ADD COLUMN `Event_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_user_priv`", } ctx := context.Background() for _, sql := range sqls { @@ -773,15 +773,15 @@ func upgradeToVer14(s Session, ver int64) { return } sqls := []string{ - "ALTER TABLE mysql.db ADD COLUMN `References_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Grant_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Create_tmp_table_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Alter_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Lock_tables_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_tmp_table_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Create_view_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Lock_tables_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Show_view_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_view_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Create_routine_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Show_view_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Alter_routine_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_routine_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Event_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Execute_priv`", - "ALTER TABLE mysql.db ADD COLUMN `Trigger_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Event_priv`", + "ALTER TABLE mysql.db ADD COLUMN `References_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Grant_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Create_tmp_table_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Alter_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Lock_tables_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_tmp_table_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Create_view_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Lock_tables_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Show_view_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_view_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Create_routine_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Show_view_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Alter_routine_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_routine_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Event_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Execute_priv`", + "ALTER TABLE mysql.db ADD COLUMN `Trigger_priv` ENUM('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Event_priv`", } ctx := context.Background() for _, sql := range sqls { @@ -810,7 +810,7 @@ func upgradeToVer16(s Session, ver int64) { if ver >= version16 { return } - doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `cm_sketch` blob", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `cm_sketch` BLOB", infoschema.ErrColumnExists) } func upgradeToVer17(s Session, ver int64) { @@ -824,7 +824,7 @@ func upgradeToVer18(s Session, ver int64) { if ver >= version18 { return } - doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `tot_col_size` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `tot_col_size` BIGINT(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } func upgradeToVer19(s Session, ver int64) { @@ -858,14 +858,14 @@ func upgradeToVer22(s Session, ver int64) { if ver >= version22 { return } - doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `stats_ver` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `stats_ver` BIGINT(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } func upgradeToVer23(s Session, ver int64) { if ver >= version23 { return } - doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `flag` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `flag` BIGINT(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } // writeSystemTZ writes system timezone info into mysql.tidb @@ -912,7 +912,7 @@ func upgradeToVer27(s Session, ver int64) { if ver >= version27 { return } - doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `correlation` double NOT NULL DEFAULT 0", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `correlation` DOUBLE NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } func upgradeToVer28(s Session, ver int64) { @@ -927,9 +927,9 @@ func upgradeToVer29(s Session, ver int64) { if ver != version28 { return } - doReentrantDDL(s, "ALTER TABLE mysql.bind_info change create_time create_time timestamp(3)") - doReentrantDDL(s, "ALTER TABLE mysql.bind_info change update_time update_time timestamp(3)") - doReentrantDDL(s, "ALTER TABLE mysql.bind_info add index sql_index (original_sql(1024),default_db(1024))", ddl.ErrDupKeyName) + doReentrantDDL(s, "ALTER TABLE mysql.bind_info CHANGE create_time create_time TIMESTAMP(3)") + doReentrantDDL(s, "ALTER TABLE mysql.bind_info CHANGE update_time update_time TIMESTAMP(3)") + doReentrantDDL(s, "ALTER TABLE mysql.bind_info ADD INDEX sql_index (original_sql(1024),default_db(1024))", ddl.ErrDupKeyName) } func upgradeToVer30(s Session, ver int64) { @@ -943,7 +943,7 @@ func upgradeToVer31(s Session, ver int64) { if ver >= version31 { return } - doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `last_analyze_pos` blob default null", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `last_analyze_pos` BLOB DEFAULT NULL", infoschema.ErrColumnExists) } func upgradeToVer32(s Session, ver int64) { @@ -982,7 +982,7 @@ func upgradeToVer36(s Session, ver int64) { } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Shutdown_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) // A root user will have those privileges after upgrading. - mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Shutdown_priv='Y' where Super_priv='Y'") + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Shutdown_priv='Y' WHERE Super_priv='Y'") mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_tmp_table_priv='Y',Lock_tables_priv='Y',Create_routine_priv='Y',Alter_routine_priv='Y',Event_priv='Y' WHERE Super_priv='Y'") } @@ -1013,8 +1013,8 @@ func upgradeToVer39(s Session, ver int64) { } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Reload_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `File_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) - mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Reload_priv='Y' where Super_priv='Y'") - mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET File_priv='Y' where Super_priv='Y'") + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Reload_priv='Y' WHERE Super_priv='Y'") + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET File_priv='Y' WHERE Super_priv='Y'") } func writeNewCollationParameter(s Session, flag bool) { @@ -1055,8 +1055,8 @@ func upgradeToVer42(s Session, ver int64) { if ver >= version42 { return } - doReentrantDDL(s, "ALTER TABLE mysql.expr_pushdown_blacklist ADD COLUMN `store_type` char(100) NOT NULL DEFAULT 'tikv,tiflash,tidb'", infoschema.ErrColumnExists) - doReentrantDDL(s, "ALTER TABLE mysql.expr_pushdown_blacklist ADD COLUMN `reason` varchar(200)", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.expr_pushdown_blacklist ADD COLUMN `store_type` CHAR(100) NOT NULL DEFAULT 'tikv,tiflash,tidb'", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.expr_pushdown_blacklist ADD COLUMN `reason` VARCHAR(200)", infoschema.ErrColumnExists) writeDefaultExprPushDownBlacklist(s) } @@ -1091,7 +1091,7 @@ func upgradeToVer45(s Session, ver int64) { return } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Config_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) - mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Config_priv='Y' where Super_priv='Y'") + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Config_priv='Y' WHERE Super_priv='Y'") } // In v3.1.1, we wrongly replace the context of upgradeToVer39 with upgradeToVer44. If we upgrade from v3.1.1 to a newer version, @@ -1102,8 +1102,8 @@ func upgradeToVer46(s Session, ver int64) { } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Reload_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `File_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) - mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Reload_priv='Y' where Super_priv='Y'") - mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET File_priv='Y' where Super_priv='Y'") + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Reload_priv='Y' WHERE Super_priv='Y'") + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET File_priv='Y' WHERE Super_priv='Y'") } func upgradeToVer47(s Session, ver int64) { diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 66a2b405fff64..f5cb3046e1dbe 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -815,6 +815,145 @@ func (s *testPessimisticSuite) TestBatchPointGetWriteConflict(c *C) { tk1.MustExec("commit") } +func (s *testPessimisticSuite) TestPessimisticSerializable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test") + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk1.MustExec("use test") + + tk.MustExec("set tidb_txn_mode = 'pessimistic'") + tk1.MustExec("set tidb_txn_mode = 'pessimistic'") + + tk.MustExec("drop table if exists test;") + tk.MustExec("create table test (id int not null primary key, value int);") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("set tidb_skip_isolation_level_check = 1") + tk1.MustExec("set tidb_skip_isolation_level_check = 1") + tk.MustExec("set tx_isolation = 'SERIALIZABLE'") + tk1.MustExec("set tx_isolation = 'SERIALIZABLE'") + + // Predicate-Many-Preceders (PMP) + tk.MustExec("begin") + tk1.MustExec("begin") + tk.MustQuery("select * from test where value = 30;").Check(testkit.Rows()) + tk1.MustExec("insert into test (id, value) values(3, 30);") + tk1.MustExec("commit") + tk.MustQuery("select * from test where mod(value, 3) = 0;").Check(testkit.Rows()) + tk.MustExec("commit") + + tk.MustExec("truncate table test;") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("begin;") + tk1.MustExec("begin;") + tk.MustExec("update test set value = value + 10;") + + var wg sync.WaitGroup + wg.Add(1) + go func() { + tk1.ExecToErr("delete from test where value = 20;") + wg.Done() + }() + tk.MustExec("commit;") + wg.Wait() + tk1.MustExec("rollback;") + + // Lost Update (P4) + tk.MustExec("truncate table test;") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("begin;") + tk1.MustExec("begin;") + tk.MustQuery("select * from test where id = 1;").Check(testkit.Rows("1 10")) + tk1.MustQuery("select * from test where id = 1;").Check(testkit.Rows("1 10")) + tk.MustExec("update test set value = 11 where id = 1;") + + wg.Add(1) + go func() { + tk1.ExecToErr("update test set value = 11 where id = 1;") + wg.Done() + }() + tk.MustExec("commit;") + wg.Wait() + tk1.MustExec("rollback;") + + // Read Skew (G-single) + tk.MustExec("truncate table test;") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("begin;") + tk1.MustExec("begin;") + tk.MustQuery("select * from test where id = 1;").Check(testkit.Rows("1 10")) + tk1.MustQuery("select * from test where id = 1;").Check(testkit.Rows("1 10")) + tk1.MustQuery("select * from test where id = 2;").Check(testkit.Rows("2 20")) + tk1.MustExec("update test set value = 12 where id = 1;") + tk1.MustExec("update test set value = 18 where id = 1;") + tk1.MustExec("commit;") + tk.MustQuery("select * from test where id = 2;").Check(testkit.Rows("2 20")) + tk.MustExec("commit;") + + tk.MustExec("truncate table test;") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("begin;") + tk1.MustExec("begin;") + tk.MustQuery("select * from test where mod(value, 5) = 0;").Check(testkit.Rows("1 10", "2 20")) + tk1.MustExec("update test set value = 12 where value = 10;") + tk1.MustExec("commit;") + tk.MustQuery("select * from test where mod(value, 3) = 0;").Check(testkit.Rows()) + tk.MustExec("commit;") + + tk.MustExec("truncate table test;") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("begin;") + tk1.MustExec("begin;") + tk.MustQuery("select * from test where id = 1;").Check(testkit.Rows("1 10")) + tk1.MustQuery("select * from test;").Check(testkit.Rows("1 10", "2 20")) + tk1.MustExec("update test set value = 12 where id = 1;") + tk1.MustExec("update test set value = 18 where id = 1;") + tk1.MustExec("commit;") + tk.ExecToErr("delete from test where value = 20;") + tk.MustExec("rollback;") + + // Write Skew (G2-item) + tk.MustExec("truncate table test;") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("begin;") + tk1.MustExec("begin;") + tk.MustQuery("select * from test where id in (1,2);").Check(testkit.Rows("1 10", "2 20")) + tk1.MustQuery("select * from test where id in (1,2);").Check(testkit.Rows("1 10", "2 20")) + tk1.MustExec("update test set value = 11 where id = 1;") + tk1.MustExec("update test set value = 21 where id = 2;") + tk.MustExec("commit;") + tk1.MustExec("commit;") + tk.MustQuery("select * from test;").Check(testkit.Rows("1 11", "2 21")) + + // Anti-Dependency Cycles (G2) + tk.MustExec("truncate table test;") + tk.MustExec("insert into test (id, value) values (1, 10);") + tk.MustExec("insert into test (id, value) values (2, 20);") + + tk.MustExec("begin;") + tk1.MustExec("begin;") + tk.MustQuery("select * from test where mod(value, 3) = 0;").Check(testkit.Rows()) + tk1.MustQuery("select * from test where mod(value, 5) = 0;").Check(testkit.Rows("1 10", "2 20")) + tk.MustExec("insert into test (id, value) values(3, 30);") + tk1.MustExec("insert into test (id, value) values(4, 60);") + tk.MustExec("commit;") + tk1.MustExec("commit;") + tk.MustQuery("select * from test where mod(value, 3) = 0;").Check(testkit.Rows("3 30", "4 60")) +} + func (s *testPessimisticSuite) TestPessimisticReadCommitted(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test") @@ -1824,3 +1963,33 @@ func (s *testPessimisticSuite) TestSelectForUpdateConflictRetry(c *C) { // should increase c.Assert(tk3LastTs, Greater, tk2LastTS) } + +func (s *testPessimisticSuite) TestAsyncCommitWithSchemaChange(c *C) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.Enable = true + }) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforeSchemaCheck", "return"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforeSchemaCheck"), IsNil) + }() + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists tk") + tk.MustExec("create table tk (c1 int primary key, c2 int)") + tk.MustExec("insert into tk values(1,1),(2,2)") + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk3 := testkit.NewTestKitWithInit(c, s.store) + + // The txn tk writes something but with failpoint the primary key is not committed. + tk.MustExec("begin optimistic") + // Change the schema version. + tk2.MustExec("alter table tk add column c3 int after c2") + tk.MustExec("insert into tk values(3, 3)") + tk.MustExec("commit") + + // Trigger the recovery process, the left locks should not be committed. + tk3.MustExec("begin") + tk3.MustQuery("select * from tk").Check(testkit.Rows("1 1 ", "2 2 ")) + tk3.MustExec("rollback") +} diff --git a/session/session.go b/session/session.go index 677e19a540704..6a8cbe1178882 100644 --- a/session/session.go +++ b/session/session.go @@ -509,8 +509,10 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { } mapper := s.GetSessionVars().TxnCtx.TableDeltaMap if s.statsCollector != nil && mapper != nil { - for id, item := range mapper { - s.statsCollector.Update(id, item.Delta, item.Count, &item.ColSize) + for _, item := range mapper { + if item.TableID > 0 { + s.statsCollector.Update(item.TableID, item.Delta, item.Count, &item.ColSize) + } } } return nil @@ -786,6 +788,7 @@ func (s *session) ExecRestrictedSQLWithContext(ctx context.Context, sql string) se.sessionVars.PartitionPruneMode.Store(prePruneMode) s.sysSessionPool().Put(tmp) }() + // for analyze stmt we need let worker session follow user session that executing stmt. se.sessionVars.PartitionPruneMode.Store(s.sessionVars.PartitionPruneMode.Load()) metrics.SessionRestrictedSQLCounter.Inc() @@ -1005,6 +1008,12 @@ func (s *session) SetGlobalSysVar(name, value string) error { return err } } + if name == variable.TiDBPartitionPruneMode && value == string(variable.DynamicOnly) { + err := s.ensureFullGlobalStats() + if err != nil { + return err + } + } var sVal string var err error sVal, err = variable.ValidateSetSystemVar(s.sessionVars, name, value, variable.ScopeGlobal) @@ -1019,6 +1028,20 @@ func (s *session) SetGlobalSysVar(name, value string) error { return err } +func (s *session) ensureFullGlobalStats() error { + rows, _, err := s.ExecRestrictedSQL(`select count(1) from information_schema.tables t where t.create_options = 'partitioned' + and not exists (select 1 from mysql.stats_meta m where m.table_id = t.tidb_table_id)`) + if err != nil { + return err + } + row := rows[0] + count := row.GetInt64(0) + if count > 0 { + return errors.New("need analyze all partition table in 'static-collect-dynamic' mode before switch to 'dynamic-only'") + } + return nil +} + func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) ([]ast.StmtNode, []error, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.ParseSQL", opentracing.ChildOf(span.Context())) @@ -1225,6 +1248,7 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. se.StmtCommit() } } + err = finishStmt(ctx, se, err, s) } if rs != nil { return &execStmtResult{ @@ -1234,8 +1258,6 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. }, err } - err = finishStmt(ctx, se, err, s) - // If it is not a select statement, we record its slow log here, // then it could include the transaction commit time. s.(*executor.ExecStmt).FinishExecuteStmt(origTxnCtx.StartTS, err == nil, false) @@ -2303,7 +2325,7 @@ func logQuery(query string, vars *variable.SessionVars) { zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), zap.Uint64("txnStartTS", vars.TxnCtx.StartTS), zap.Uint64("forUpdateTS", vars.TxnCtx.GetForUpdateTS()), - zap.Bool("isReadConsistency", vars.IsReadConsistencyTxn()), + zap.Bool("isReadConsistency", vars.IsIsolation(ast.ReadCommitted)), zap.String("current_db", vars.CurrentDB), zap.String("txn_mode", vars.GetReadableTxnMode()), zap.String("sql", query)) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 3bb947b2a9bfa..5648433063088 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -215,7 +215,7 @@ func (tc *TransactionContext) CollectUnchangedRowKeys(buf []kv.Key) []kv.Key { } // UpdateDeltaForTable updates the delta info for some table. -func (tc *TransactionContext) UpdateDeltaForTable(physicalTableID int64, delta int64, count int64, colSize map[int64]int64) { +func (tc *TransactionContext) UpdateDeltaForTable(logicalTableID, physicalTableID int64, delta int64, count int64, colSize map[int64]int64, saveAsLogicalTblID bool) { if tc.TableDeltaMap == nil { tc.TableDeltaMap = make(map[int64]TableDelta) } @@ -225,6 +225,10 @@ func (tc *TransactionContext) UpdateDeltaForTable(physicalTableID int64, delta i } item.Delta += delta item.Count += count + item.TableID = physicalTableID + if saveAsLogicalTblID { + item.TableID = logicalTableID + } for key, val := range colSize { item.ColSize[key] += val } @@ -837,6 +841,7 @@ func NewSessionVars() *SessionVars { ShardAllocateStep: DefTiDBShardAllocateStep, EnableChangeColumnType: DefTiDBChangeColumnType, EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, + PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ @@ -1035,10 +1040,10 @@ func (s *SessionVars) IsAutocommit() bool { return s.GetStatusFlag(mysql.ServerStatusAutocommit) } -// IsReadConsistencyTxn if true it means the transaction is an read consistency (read committed) transaction. -func (s *SessionVars) IsReadConsistencyTxn() bool { +// IsIsolation if true it means the transaction is at that isolation level. +func (s *SessionVars) IsIsolation(isolation string) bool { if s.TxnCtx.Isolation != "" { - return s.TxnCtx.Isolation == ast.ReadCommitted + return s.TxnCtx.Isolation == isolation } if s.txnIsolationLevelOneShot.state == oneShotUse { s.TxnCtx.Isolation = s.txnIsolationLevelOneShot.value @@ -1046,7 +1051,7 @@ func (s *SessionVars) IsReadConsistencyTxn() bool { if s.TxnCtx.Isolation == "" { s.TxnCtx.Isolation, _ = s.GetSystemVar(TxnIsolation) } - return s.TxnCtx.Isolation == ast.ReadCommitted + return s.TxnCtx.Isolation == isolation } // SetTxnIsolationLevelOneShotStateForNextTxn sets the txnIsolationLevelOneShot.state for next transaction. @@ -1064,7 +1069,7 @@ func (s *SessionVars) SetTxnIsolationLevelOneShotStateForNextTxn() { // IsPessimisticReadConsistency if true it means the statement is in an read consistency pessimistic transaction. func (s *SessionVars) IsPessimisticReadConsistency() bool { - return s.TxnCtx.IsPessimistic && s.IsReadConsistencyTxn() + return s.TxnCtx.IsPessimistic && s.IsIsolation(ast.ReadCommitted) } // GetNextPreparedStmtID generates and returns the next session scope prepared statement id. @@ -1555,6 +1560,7 @@ type TableDelta struct { Count int64 ColSize map[int64]int64 InitTime time.Time // InitTime is the time that this delta is generated. + TableID int64 } // ConcurrencyUnset means the value the of the concurrency related variable is unset. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index a613745326521..bb1314269ba05 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -14,6 +14,7 @@ package variable import ( + "math" "strconv" "strings" "sync" @@ -40,24 +41,26 @@ const ( // ScopeSession means the system variable can only be changed in current session. ScopeSession ScopeFlag = 1 << 1 + // TypeUnknown for not yet defined + TypeUnknown TypeFlag = 0 // TypeBool for boolean - TypeBool TypeFlag = 0 + TypeBool TypeFlag = 1 // TypeInt for integer - TypeInt TypeFlag = 1 + TypeInt TypeFlag = 2 // TypeLong for Long - TypeLong TypeFlag = 2 + TypeLong TypeFlag = 3 // TypeLongLong for LongLong - TypeLongLong TypeFlag = 3 + TypeLongLong TypeFlag = 4 // TypeStr for String - TypeStr TypeFlag = 4 + TypeStr TypeFlag = 5 // TypeEnum for Enum - TypeEnum TypeFlag = 5 + TypeEnum TypeFlag = 6 // TypeSet for Set - TypeSet TypeFlag = 6 + TypeSet TypeFlag = 7 // TypeDouble for Double - TypeDouble TypeFlag = 7 + TypeDouble TypeFlag = 8 // TypeUnsigned for Unsigned integer - TypeUnsigned TypeFlag = 8 + TypeUnsigned TypeFlag = 9 ) // SysVar is for system variable. @@ -78,7 +81,13 @@ type SysVar struct { MinValue int64 // MaxValue will automatically be validated when specified (optional) - MaxValue int64 + MaxValue uint64 + + // AutoConvertNegativeBool applies to boolean types (optional) + AutoConvertNegativeBool bool + + // AutoConvertOutOfRange applies to int and unsigned types. + AutoConvertOutOfRange bool } var sysVars map[string]*SysVar @@ -152,57 +161,57 @@ func BoolToInt32(b bool) int32 { // we only support MySQL now var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "gtid_mode", Value: "OFF", Type: TypeBool}, - {Scope: ScopeGlobal, Name: FlushTime, Value: "0"}, + {Scope: ScopeGlobal, Name: FlushTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "performance_schema_max_mutex_classes", Value: "200"}, - {Scope: ScopeGlobal | ScopeSession, Name: LowPriorityUpdates, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: LowPriorityUpdates, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: SessionTrackGtids, Value: "OFF"}, {Scope: ScopeGlobal | ScopeSession, Name: "ndbinfo_max_rows", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "ndb_index_stat_option", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: OldPasswords, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: OldPasswords, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 2, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "innodb_version", Value: "5.6.25"}, - {Scope: ScopeGlobal, Name: MaxConnections, Value: "151"}, - {Scope: ScopeGlobal | ScopeSession, Name: BigTables, Value: "0"}, + {Scope: ScopeGlobal, Name: MaxConnections, Value: "151", Type: TypeUnsigned, MinValue: 1, MaxValue: 100000, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal | ScopeSession, Name: BigTables, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "skip_external_locking", Value: "1"}, {Scope: ScopeNone, Name: "innodb_sync_array_size", Value: "1"}, {Scope: ScopeSession, Name: "rand_seed2", Value: ""}, - {Scope: ScopeGlobal, Name: ValidatePasswordCheckUserName, Value: "0"}, - {Scope: ScopeGlobal, Name: "validate_password_number_count", Value: "1"}, + {Scope: ScopeGlobal, Name: ValidatePasswordCheckUserName, Value: "0", Type: TypeBool}, + {Scope: ScopeGlobal, Name: ValidatePasswordNumberCount, Value: "1", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true}, {Scope: ScopeSession, Name: "gtid_next", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLSelectLimit, Value: "18446744073709551615"}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLSelectLimit, Value: "18446744073709551615", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal, Name: "ndb_show_foreign_key_mock_tables", Value: ""}, {Scope: ScopeNone, Name: "multi_range_count", Value: "256"}, - {Scope: ScopeGlobal | ScopeSession, Name: DefaultWeekFormat, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: DefaultWeekFormat, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 7, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal | ScopeSession, Name: "binlog_error_action", Value: "IGNORE_ERROR"}, {Scope: ScopeGlobal | ScopeSession, Name: "default_storage_engine", Value: "InnoDB"}, {Scope: ScopeNone, Name: "ft_query_expansion_limit", Value: "20"}, - {Scope: ScopeGlobal, Name: MaxConnectErrors, Value: "100"}, - {Scope: ScopeGlobal, Name: SyncBinlog, Value: "0"}, + {Scope: ScopeGlobal, Name: MaxConnectErrors, Value: "100", Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: SyncBinlog, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 4294967295, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "max_digest_length", Value: "1024"}, {Scope: ScopeNone, Name: "innodb_force_load_corrupted", Value: "0"}, {Scope: ScopeNone, Name: "performance_schema_max_table_handles", Value: "4000"}, - {Scope: ScopeGlobal, Name: InnodbFastShutdown, Value: "1"}, + {Scope: ScopeGlobal, Name: InnodbFastShutdown, Value: "1", Type: TypeUnsigned, MinValue: 0, MaxValue: 2, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "ft_max_word_len", Value: "84"}, {Scope: ScopeGlobal, Name: "log_backward_compatible_user_definitions", Value: ""}, {Scope: ScopeNone, Name: "lc_messages_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/"}, {Scope: ScopeGlobal, Name: "ft_boolean_syntax", Value: "+ -><()~*:\"\"&|"}, - {Scope: ScopeGlobal, Name: TableDefinitionCache, Value: "-1"}, - {Scope: ScopeNone, Name: SkipNameResolve, Value: "0"}, + {Scope: ScopeGlobal, Name: TableDefinitionCache, Value: "-1", Type: TypeUnsigned, MinValue: 400, MaxValue: 524288, AutoConvertOutOfRange: true}, + {Scope: ScopeNone, Name: SkipNameResolve, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "performance_schema_max_file_handles", Value: "32768"}, {Scope: ScopeSession, Name: "transaction_allow_batching", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: SQLModeVar, Value: mysql.DefaultSQLMode}, {Scope: ScopeNone, Name: "performance_schema_max_statement_classes", Value: "168"}, {Scope: ScopeGlobal, Name: "server_id", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_flushing_avg_loops", Value: "30"}, - {Scope: ScopeGlobal | ScopeSession, Name: TmpTableSize, Value: "16777216"}, + {Scope: ScopeGlobal | ScopeSession, Name: TmpTableSize, Value: "16777216", Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal, Name: "innodb_max_purge_lag", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "preload_buffer_size", Value: "32768"}, - {Scope: ScopeGlobal, Name: CheckProxyUsers, Value: "0"}, + {Scope: ScopeGlobal, Name: CheckProxyUsers, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "have_query_cache", Value: "YES"}, {Scope: ScopeGlobal, Name: "innodb_flush_log_at_timeout", Value: "1"}, {Scope: ScopeGlobal, Name: "innodb_max_undo_log_size", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "range_alloc_block_size", Value: "4096"}, - {Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10"}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxExecutionTime, Value: "0"}, + {Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal | ScopeSession, Name: MaxExecutionTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal | ScopeSession, Name: CollationServer, Value: mysql.DefaultCollationName}, {Scope: ScopeNone, Name: "have_rtree_keys", Value: "YES"}, {Scope: ScopeGlobal, Name: "innodb_old_blocks_pct", Value: "37"}, @@ -213,8 +222,8 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_ft_sort_pll_degree", Value: "2"}, {Scope: ScopeNone, Name: "thread_stack", Value: "262144"}, {Scope: ScopeGlobal, Name: "relay_log_info_repository", Value: "FILE"}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: "1"}, - {Scope: ScopeGlobal, Name: SuperReadOnly, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: "1", Type: TypeBool}, + {Scope: ScopeGlobal, Name: SuperReadOnly, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "max_delayed_threads", Value: "20"}, {Scope: ScopeNone, Name: "protocol_version", Value: "10"}, {Scope: ScopeGlobal | ScopeSession, Name: "new", Value: "OFF"}, @@ -222,7 +231,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_offset", Value: "-1"}, {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpAtShutdown, Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: SQLNotes, Value: "1"}, - {Scope: ScopeGlobal, Name: InnodbCmpPerIndexEnabled, Value: "0"}, + {Scope: ScopeGlobal, Name: InnodbCmpPerIndexEnabled, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: "innodb_ft_server_stopword_table", Value: ""}, {Scope: ScopeNone, Name: "performance_schema_max_file_instances", Value: "7693"}, {Scope: ScopeNone, Name: "log_output", Value: "FILE"}, @@ -232,15 +241,15 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_log_write_ahead_size", Value: ""}, {Scope: ScopeNone, Name: "innodb_log_group_home_dir", Value: "./"}, {Scope: ScopeNone, Name: "performance_schema_events_statements_history_size", Value: "10"}, - {Scope: ScopeGlobal, Name: GeneralLog, Value: "0"}, + {Scope: ScopeGlobal, Name: GeneralLog, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal, Name: "validate_password_dictionary_file", Value: ""}, - {Scope: ScopeGlobal, Name: BinlogOrderCommits, Value: "1"}, + {Scope: ScopeGlobal, Name: BinlogOrderCommits, Value: "1", Type: TypeBool}, {Scope: ScopeGlobal, Name: "key_cache_division_limit", Value: "100"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_insert_delayed_threads", Value: "20"}, {Scope: ScopeNone, Name: "performance_schema_session_connect_attrs_size", Value: "512"}, {Scope: ScopeGlobal | ScopeSession, Name: "time_zone", Value: "SYSTEM"}, {Scope: ScopeGlobal, Name: "innodb_max_dirty_pages_pct", Value: "75"}, - {Scope: ScopeGlobal, Name: InnodbFilePerTable, Value: "1"}, + {Scope: ScopeGlobal, Name: InnodbFilePerTable, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: InnodbLogCompressedPages, Value: "1"}, {Scope: ScopeNone, Name: "skip_networking", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_monitor_reset", Value: ""}, @@ -252,7 +261,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "ssl_cipher", Value: ""}, {Scope: ScopeNone, Name: "tls_version", Value: "TLSv1,TLSv1.1,TLSv1.2"}, {Scope: ScopeNone, Name: "system_time_zone", Value: "CST"}, - {Scope: ScopeGlobal, Name: InnodbPrintAllDeadlocks, Value: "0"}, + {Scope: ScopeGlobal, Name: InnodbPrintAllDeadlocks, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeNone, Name: "innodb_autoinc_lock_mode", Value: "1"}, {Scope: ScopeGlobal, Name: "key_buffer_size", Value: "8388608"}, {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: "OFF"}, @@ -263,15 +272,15 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_file_format_max", Value: "Antelope"}, {Scope: ScopeGlobal | ScopeSession, Name: "debug", Value: ""}, {Scope: ScopeGlobal, Name: "log_warnings", Value: "1"}, - {Scope: ScopeGlobal, Name: OfflineMode, Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: InnodbStrictMode, Value: "1"}, + {Scope: ScopeGlobal, Name: OfflineMode, Value: "0", Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: InnodbStrictMode, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: "innodb_rollback_segments", Value: "128"}, {Scope: ScopeGlobal | ScopeSession, Name: "join_buffer_size", Value: "262144"}, {Scope: ScopeNone, Name: "innodb_mirrored_log_groups", Value: "1"}, {Scope: ScopeGlobal, Name: "max_binlog_size", Value: "1073741824"}, {Scope: ScopeGlobal, Name: "concurrent_insert", Value: "AUTO"}, - {Scope: ScopeGlobal, Name: InnodbAdaptiveHashIndex, Value: "1"}, - {Scope: ScopeGlobal, Name: InnodbFtEnableStopword, Value: "1"}, + {Scope: ScopeGlobal, Name: InnodbAdaptiveHashIndex, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: InnodbFtEnableStopword, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: "general_log_file", Value: "/usr/local/mysql/data/localhost.log"}, {Scope: ScopeGlobal | ScopeSession, Name: InnodbSupportXA, Value: "1"}, {Scope: ScopeGlobal, Name: "innodb_compression_level", Value: "6"}, @@ -281,7 +290,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: BlockEncryptionMode, Value: "aes-128-ecb"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_length_for_sort_data", Value: "1024"}, {Scope: ScopeNone, Name: "character_set_system", Value: "utf8"}, - {Scope: ScopeGlobal | ScopeSession, Name: InteractiveTimeout, Value: "28800"}, + {Scope: ScopeGlobal | ScopeSession, Name: InteractiveTimeout, Value: "28800", Type: TypeUnsigned, MinValue: 1, MaxValue: secondsPerYear, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal, Name: InnodbOptimizeFullTextOnly, Value: "0"}, {Scope: ScopeNone, Name: "character_sets_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/charsets/"}, {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheType, Value: "OFF"}, @@ -292,17 +301,17 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "thread_concurrency", Value: "10"}, {Scope: ScopeGlobal | ScopeSession, Name: "query_prealloc_size", Value: "8192"}, {Scope: ScopeNone, Name: "relay_log_space_limit", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxUserConnections, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: MaxUserConnections, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 4294967295, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "performance_schema_max_thread_classes", Value: "50"}, {Scope: ScopeGlobal, Name: "innodb_api_trx_level", Value: "0"}, {Scope: ScopeNone, Name: "disconnect_on_expired_password", Value: "1"}, {Scope: ScopeNone, Name: "performance_schema_max_file_classes", Value: "50"}, {Scope: ScopeGlobal, Name: "expire_logs_days", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: BinlogRowQueryLogEvents, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: BinlogRowQueryLogEvents, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal, Name: "default_password_lifetime", Value: ""}, {Scope: ScopeNone, Name: "pid_file", Value: "/usr/local/mysql/data/localhost.pid"}, {Scope: ScopeNone, Name: "innodb_undo_tablespaces", Value: "0"}, - {Scope: ScopeGlobal, Name: InnodbStatusOutputLocks, Value: "0"}, + {Scope: ScopeGlobal, Name: InnodbStatusOutputLocks, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeNone, Name: "performance_schema_accounts_size", Value: "100"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_error_count", Value: "64"}, {Scope: ScopeGlobal, Name: "max_write_lock_count", Value: "18446744073709551615"}, @@ -313,25 +322,25 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_open_files", Value: "2000"}, {Scope: ScopeGlobal, Name: "innodb_spin_wait_delay", Value: "6"}, {Scope: ScopeGlobal, Name: "thread_cache_size", Value: "9"}, - {Scope: ScopeGlobal, Name: LogSlowAdminStatements, Value: "0"}, + {Scope: ScopeGlobal, Name: LogSlowAdminStatements, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "innodb_checksums", Value: "ON"}, {Scope: ScopeNone, Name: "hostname", Value: ServerHostname}, {Scope: ScopeGlobal | ScopeSession, Name: "auto_increment_offset", Value: "1"}, {Scope: ScopeNone, Name: "ft_stopword_file", Value: "(built-in)"}, {Scope: ScopeGlobal, Name: "innodb_max_dirty_pages_pct_lwm", Value: "0"}, - {Scope: ScopeGlobal, Name: LogQueriesNotUsingIndexes, Value: "0"}, + {Scope: ScopeGlobal, Name: LogQueriesNotUsingIndexes, Value: "0", Type: TypeBool}, {Scope: ScopeSession, Name: "timestamp", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheWlockInvalidate, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheWlockInvalidate, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "sql_buffer_result", Value: "OFF"}, {Scope: ScopeGlobal | ScopeSession, Name: "character_set_filesystem", Value: "binary"}, {Scope: ScopeGlobal | ScopeSession, Name: "collation_database", Value: mysql.DefaultCollationName}, - {Scope: ScopeGlobal | ScopeSession, Name: AutoIncrementIncrement, Value: strconv.FormatInt(DefAutoIncrementIncrement, 10)}, - {Scope: ScopeGlobal | ScopeSession, Name: AutoIncrementOffset, Value: strconv.FormatInt(DefAutoIncrementOffset, 10)}, + {Scope: ScopeGlobal | ScopeSession, Name: AutoIncrementIncrement, Value: strconv.FormatInt(DefAutoIncrementIncrement, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint16, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal | ScopeSession, Name: AutoIncrementOffset, Value: strconv.FormatInt(DefAutoIncrementOffset, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint16, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal | ScopeSession, Name: "max_heap_table_size", Value: "16777216"}, {Scope: ScopeGlobal | ScopeSession, Name: "div_precision_increment", Value: "4"}, {Scope: ScopeGlobal, Name: "innodb_lru_scan_depth", Value: "1024"}, {Scope: ScopeGlobal, Name: "innodb_purge_rseg_truncate_frequency", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLAutoIsNull, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLAutoIsNull, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "innodb_api_enable_binlog", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "innodb_ft_user_stopword_table", Value: ""}, {Scope: ScopeNone, Name: "server_id_bits", Value: "32"}, @@ -347,19 +356,19 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_purge_batch_size", Value: "300"}, {Scope: ScopeNone, Name: "have_profiling", Value: "NO"}, {Scope: ScopeGlobal | ScopeSession, Name: "character_set_client", Value: mysql.DefaultCharset}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpNow, Value: "0"}, - {Scope: ScopeGlobal, Name: RelayLogPurge, Value: "1"}, + {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpNow, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal, Name: RelayLogPurge, Value: "1", Type: TypeBool}, {Scope: ScopeGlobal, Name: "ndb_distribution", Value: ""}, {Scope: ScopeGlobal, Name: "myisam_data_pointer_size", Value: "6"}, {Scope: ScopeGlobal, Name: "ndb_optimization_delay", Value: ""}, {Scope: ScopeGlobal, Name: "innodb_ft_num_word_optimize", Value: "2000"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_join_size", Value: "18446744073709551615"}, - {Scope: ScopeNone, Name: CoreFile, Value: "0"}, + {Scope: ScopeNone, Name: CoreFile, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "max_seeks_for_key", Value: "18446744073709551615"}, {Scope: ScopeNone, Name: "innodb_log_buffer_size", Value: "8388608"}, {Scope: ScopeGlobal, Name: "delayed_insert_timeout", Value: "300"}, {Scope: ScopeGlobal, Name: "max_relay_log_size", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxSortLength, Value: "1024"}, + {Scope: ScopeGlobal | ScopeSession, Name: MaxSortLength, Value: "1024", Type: TypeUnsigned, MinValue: 4, MaxValue: 8388608, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "metadata_locks_hash_instances", Value: "8"}, {Scope: ScopeGlobal, Name: "ndb_eventbuffer_free_percent", Value: ""}, {Scope: ScopeNone, Name: "large_files_support", Value: "1"}, @@ -379,16 +388,16 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_api_disable_rowlock", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_adaptive_flushing_lwm", Value: "10"}, {Scope: ScopeNone, Name: "innodb_log_files_in_group", Value: "2"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadNow, Value: "0"}, + {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadNow, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeNone, Name: "performance_schema_max_rwlock_classes", Value: "40"}, {Scope: ScopeNone, Name: "binlog_gtid_simple_recovery", Value: "1"}, {Scope: ScopeNone, Name: Port, Value: "4000"}, {Scope: ScopeNone, Name: "performance_schema_digests_size", Value: "10000"}, - {Scope: ScopeGlobal | ScopeSession, Name: Profiling, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: Profiling, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "lower_case_table_names", Value: "2"}, {Scope: ScopeSession, Name: "rand_seed1", Value: ""}, {Scope: ScopeGlobal, Name: "sha256_password_proxy_users", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLQuoteShowCreate, Value: "1"}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLQuoteShowCreate, Value: "1", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "binlogging_impossible_mode", Value: "IGNORE_ERROR"}, {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheSize, Value: "1048576"}, {Scope: ScopeGlobal, Name: "innodb_stats_transient_sample_pages", Value: "8"}, @@ -414,9 +423,9 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: InnodbStatsAutoRecalc, Value: "1"}, {Scope: ScopeGlobal | ScopeSession, Name: "lc_messages", Value: "en_US"}, {Scope: ScopeGlobal | ScopeSession, Name: "bulk_insert_buffer_size", Value: "8388608"}, - {Scope: ScopeGlobal | ScopeSession, Name: BinlogDirectNonTransactionalUpdates, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: BinlogDirectNonTransactionalUpdates, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal, Name: "innodb_change_buffering", Value: "all"}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLBigSelects, Value: "1"}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLBigSelects, Value: "1", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetResults, Value: mysql.DefaultCharset}, {Scope: ScopeGlobal, Name: "innodb_max_purge_lag_delay", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "session_track_schema", Value: ""}, @@ -427,7 +436,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "read_rnd_buffer_size", Value: "262144"}, {Scope: ScopeNone, Name: "version_comment", Value: "TiDB Server (Apache License 2.0) " + versioninfo.TiDBEdition + " Edition, MySQL 5.7 compatible"}, {Scope: ScopeGlobal | ScopeSession, Name: NetWriteTimeout, Value: "60"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadAbort, Value: "0"}, + {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadAbort, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal | ScopeSession, Name: TxnIsolation, Value: "REPEATABLE-READ"}, {Scope: ScopeGlobal | ScopeSession, Name: TransactionIsolation, Value: "REPEATABLE-READ"}, {Scope: ScopeGlobal | ScopeSession, Name: "collation_connection", Value: mysql.DefaultCollationName}, @@ -444,13 +453,13 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "large_pages", Value: "OFF"}, {Scope: ScopeGlobal | ScopeSession, Name: "session_track_system_variables", Value: ""}, {Scope: ScopeGlobal, Name: "innodb_change_buffer_max_size", Value: "25"}, - {Scope: ScopeGlobal, Name: LogBinTrustFunctionCreators, Value: "0"}, + {Scope: ScopeGlobal, Name: LogBinTrustFunctionCreators, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "innodb_write_io_threads", Value: "4"}, {Scope: ScopeGlobal, Name: "mysql_native_password_proxy_users", Value: ""}, - {Scope: ScopeGlobal, Name: serverReadOnly, Value: "0"}, + {Scope: ScopeGlobal, Name: serverReadOnly, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "large_page_size", Value: "0"}, {Scope: ScopeNone, Name: "table_open_cache_instances", Value: "1"}, - {Scope: ScopeGlobal, Name: InnodbStatsPersistent, Value: "1"}, + {Scope: ScopeGlobal, Name: InnodbStatsPersistent, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal | ScopeSession, Name: "session_track_state_change", Value: ""}, {Scope: ScopeNone, Name: "optimizer_switch", Value: "index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,engine_condition_pushdown=on,index_condition_pushdown=on,mrr=on,mrr_cost_based=on,block_nested_loop=on,batched_key_access=off,materialization=on,semijoin=on,loosescan=on,firstmatch=on,subquery_materialization_cost_based=on,use_index_extensions=on"}, {Scope: ScopeGlobal, Name: "delayed_queue_size", Value: "1000"}, @@ -466,7 +475,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "ndb_index_stat_enable", Value: ""}, {Scope: ScopeGlobal, Name: "executed_gtids_compression_period", Value: ""}, {Scope: ScopeNone, Name: "time_format", Value: "%H:%i:%s"}, - {Scope: ScopeGlobal | ScopeSession, Name: OldAlterTable, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: OldAlterTable, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "long_query_time", Value: "10.000000"}, {Scope: ScopeNone, Name: "innodb_use_native_aio", Value: "0"}, {Scope: ScopeGlobal, Name: "log_throttle_queries_not_using_indexes", Value: "0"}, @@ -474,13 +483,13 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "innodb_api_enable_mdl", Value: "0"}, {Scope: ScopeGlobal, Name: "binlog_cache_size", Value: "32768"}, {Scope: ScopeGlobal, Name: "innodb_compression_pad_pct_max", Value: "50"}, - {Scope: ScopeGlobal, Name: InnodbCommitConcurrency, Value: "0"}, + {Scope: ScopeGlobal, Name: InnodbCommitConcurrency, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 1000, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "ft_min_word_len", Value: "4"}, {Scope: ScopeGlobal, Name: EnforceGtidConsistency, Value: "OFF"}, {Scope: ScopeGlobal, Name: SecureAuth, Value: "1"}, {Scope: ScopeNone, Name: "max_tmp_tables", Value: "32"}, - {Scope: ScopeGlobal, Name: InnodbRandomReadAhead, Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: UniqueChecks, Value: "1"}, + {Scope: ScopeGlobal, Name: InnodbRandomReadAhead, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal | ScopeSession, Name: UniqueChecks, Value: "1", Type: TypeBool}, {Scope: ScopeGlobal, Name: "internal_tmp_disk_storage_engine", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "myisam_repair_threads", Value: "1"}, {Scope: ScopeGlobal, Name: "ndb_eventbuffer_max_alloc", Value: ""}, @@ -492,14 +501,14 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "lock_wait_timeout", Value: "31536000"}, {Scope: ScopeGlobal | ScopeSession, Name: "read_buffer_size", Value: "131072"}, {Scope: ScopeNone, Name: "innodb_read_io_threads", Value: "4"}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxSpRecursionDepth, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: MaxSpRecursionDepth, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 255, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "ignore_builtin_innodb", Value: "0"}, {Scope: ScopeGlobal, Name: "slow_query_log_file", Value: "/usr/local/mysql/data/localhost-slow.log"}, {Scope: ScopeGlobal, Name: "innodb_thread_sleep_delay", Value: "10000"}, {Scope: ScopeNone, Name: "license", Value: "Apache License 2.0"}, {Scope: ScopeGlobal, Name: "innodb_ft_aux_table", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLWarnings, Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: KeepFilesOnCreate, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLWarnings, Value: "0", Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: KeepFilesOnCreate, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "innodb_data_file_path", Value: "ibdata1:12M:autoextend"}, {Scope: ScopeNone, Name: "performance_schema_setup_actors_size", Value: "100"}, {Scope: ScopeNone, Name: "innodb_additional_mem_pool_size", Value: "8388608"}, @@ -510,7 +519,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "performance_schema_max_rwlock_instances", Value: "9102"}, {Scope: ScopeGlobal, Name: "table_open_cache", Value: "2000"}, {Scope: ScopeNone, Name: "performance_schema_events_stages_history_long_size", Value: "10000"}, - {Scope: ScopeGlobal | ScopeSession, Name: AutoCommit, Value: "1"}, + {Scope: ScopeGlobal | ScopeSession, Name: AutoCommit, Value: "1", Type: TypeBool}, {Scope: ScopeSession, Name: "insert_id", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "default_tmp_storage_engine", Value: "InnoDB"}, {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_search_depth", Value: "62"}, @@ -533,7 +542,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: "innodb_create_intrinsic", Value: ""}, {Scope: ScopeGlobal, Name: "gtid_executed_compression_period", Value: ""}, {Scope: ScopeGlobal, Name: "ndb_log_empty_epochs", Value: ""}, - {Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10)}, + {Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10), Type: TypeInt, MinValue: -1, MaxValue: 1048576, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "have_geometry", Value: "YES"}, {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_max_mem_size", Value: "16384"}, {Scope: ScopeGlobal | ScopeSession, Name: "net_retry_count", Value: "10"}, @@ -547,12 +556,12 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "completion_type", Value: "NO_CHAIN"}, {Scope: ScopeGlobal, Name: "binlog_checksum", Value: "CRC32"}, {Scope: ScopeNone, Name: "report_port", Value: "3306"}, - {Scope: ScopeGlobal | ScopeSession, Name: ShowOldTemporals, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: ShowOldTemporals, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal, Name: "query_cache_limit", Value: "1048576"}, {Scope: ScopeGlobal, Name: "innodb_buffer_pool_size", Value: "134217728"}, - {Scope: ScopeGlobal, Name: InnodbAdaptiveFlushing, Value: "1"}, + {Scope: ScopeGlobal, Name: InnodbAdaptiveFlushing, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeNone, Name: "datadir", Value: "/usr/local/mysql/data/"}, - {Scope: ScopeGlobal | ScopeSession, Name: WaitTimeout, Value: strconv.FormatInt(DefWaitTimeout, 10)}, + {Scope: ScopeGlobal | ScopeSession, Name: WaitTimeout, Value: strconv.FormatInt(DefWaitTimeout, 10), Type: TypeUnsigned, MinValue: 0, MaxValue: 31536000, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal, Name: "innodb_monitor_enable", Value: ""}, {Scope: ScopeNone, Name: "date_format", Value: "%Y-%m-%d"}, {Scope: ScopeGlobal, Name: "innodb_buffer_pool_filename", Value: "ib_buffer_pool"}, @@ -566,14 +575,14 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "basedir", Value: "/usr/local/mysql"}, {Scope: ScopeGlobal, Name: "innodb_old_blocks_time", Value: "1000"}, {Scope: ScopeGlobal, Name: "innodb_stats_method", Value: "nulls_equal"}, - {Scope: ScopeGlobal | ScopeSession, Name: InnodbLockWaitTimeout, Value: strconv.FormatInt(DefInnodbLockWaitTimeout, 10)}, - {Scope: ScopeGlobal, Name: LocalInFile, Value: "1"}, + {Scope: ScopeGlobal | ScopeSession, Name: InnodbLockWaitTimeout, Value: strconv.FormatInt(DefInnodbLockWaitTimeout, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: 1073741824, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: LocalInFile, Value: "1", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "myisam_stats_method", Value: "nulls_unequal"}, {Scope: ScopeNone, Name: "version_compile_os", Value: "osx10.8"}, {Scope: ScopeNone, Name: "relay_log_recovery", Value: "0"}, {Scope: ScopeNone, Name: "old", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: InnodbTableLocks, Value: "1"}, - {Scope: ScopeNone, Name: PerformanceSchema, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: InnodbTableLocks, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeNone, Name: PerformanceSchema, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "myisam_recover_options", Value: "OFF"}, {Scope: ScopeGlobal | ScopeSession, Name: NetBufferLength, Value: "16384"}, {Scope: ScopeGlobal | ScopeSession, Name: "binlog_row_image", Value: "FULL"}, @@ -582,7 +591,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "myisam_max_sort_file_size", Value: "9223372036853727232"}, {Scope: ScopeNone, Name: "back_log", Value: "80"}, {Scope: ScopeNone, Name: "lower_case_file_system", Value: "1"}, - {Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024"}, + {Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024", AutoConvertOutOfRange: true}, {Scope: ScopeSession, Name: "pseudo_thread_id", Value: ""}, {Scope: ScopeNone, Name: "socket", Value: "/tmp/myssock"}, {Scope: ScopeNone, Name: "have_dynamic_loading", Value: "YES"}, @@ -590,11 +599,11 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_undo_logs", Value: "128"}, {Scope: ScopeNone, Name: "performance_schema_max_cond_instances", Value: "3504"}, {Scope: ScopeGlobal, Name: "delayed_insert_limit", Value: "100"}, - {Scope: ScopeGlobal, Name: Flush, Value: "0"}, + {Scope: ScopeGlobal, Name: Flush, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "eq_range_index_dive_limit", Value: "10"}, {Scope: ScopeNone, Name: "performance_schema_events_stages_history_size", Value: "10"}, {Scope: ScopeGlobal | ScopeSession, Name: "character_set_connection", Value: mysql.DefaultCharset}, - {Scope: ScopeGlobal, Name: MyISAMUseMmap, Value: "0"}, + {Scope: ScopeGlobal, Name: MyISAMUseMmap, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal | ScopeSession, Name: "ndb_join_pushdown", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetServer, Value: mysql.DefaultCharset}, {Scope: ScopeGlobal, Name: "validate_password_special_char_count", Value: "1"}, @@ -602,27 +611,27 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "ndbinfo_show_hidden", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "net_read_timeout", Value: "30"}, {Scope: ScopeNone, Name: "innodb_page_size", Value: "16384"}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxAllowedPacket, Value: "67108864"}, + {Scope: ScopeGlobal | ScopeSession, Name: MaxAllowedPacket, Value: "67108864", Type: TypeUnsigned, MinValue: 1024, MaxValue: MaxOfMaxAllowedPacket, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: "innodb_log_file_size", Value: "50331648"}, {Scope: ScopeGlobal, Name: "sync_relay_log_info", Value: "10000"}, {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_limit", Value: "1"}, {Scope: ScopeNone, Name: "innodb_ft_max_token_size", Value: "84"}, - {Scope: ScopeGlobal, Name: "validate_password_length", Value: "8"}, + {Scope: ScopeGlobal, Name: ValidatePasswordLength, Value: "8", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true}, {Scope: ScopeGlobal, Name: "ndb_log_binlog_index", Value: ""}, {Scope: ScopeGlobal, Name: "innodb_api_bk_commit_interval", Value: "5"}, {Scope: ScopeNone, Name: "innodb_undo_directory", Value: "."}, {Scope: ScopeNone, Name: "bind_address", Value: "*"}, {Scope: ScopeGlobal, Name: "innodb_sync_spin_loops", Value: "30"}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLSafeUpdates, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLSafeUpdates, Value: "0", Type: TypeBool}, {Scope: ScopeNone, Name: "tmpdir", Value: "/var/tmp/"}, {Scope: ScopeGlobal, Name: "innodb_thread_concurrency", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_buffer_pool_dump_pct", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "lc_time_names", Value: "en_US"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_statement_time", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: EndMakersInJSON, Value: "0"}, - {Scope: ScopeGlobal, Name: AvoidTemporalUpgrade, Value: "0"}, + {Scope: ScopeGlobal | ScopeSession, Name: EndMakersInJSON, Value: "0", Type: TypeBool}, + {Scope: ScopeGlobal, Name: AvoidTemporalUpgrade, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal, Name: "key_cache_age_threshold", Value: "300"}, - {Scope: ScopeGlobal, Name: InnodbStatusOutput, Value: "0"}, + {Scope: ScopeGlobal, Name: InnodbStatusOutput, Value: "0", Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeSession, Name: "identity", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "min_examined_row_limit", Value: "0"}, {Scope: ScopeGlobal, Name: "sync_frm", Value: "ON"}, @@ -630,24 +639,24 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: WarningCount, Value: "0"}, {Scope: ScopeSession, Name: ErrorCount, Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "information_schema_stats_expiry", Value: "86400"}, - {Scope: ScopeGlobal, Name: "thread_pool_size", Value: "16"}, - {Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: "ON"}, + {Scope: ScopeGlobal, Name: ThreadPoolSize, Value: "16", Type: TypeUnsigned, MinValue: 1, MaxValue: 64, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: "ON", Type: TypeBool}, /* TiDB specific variables */ {Scope: ScopeSession, Name: TiDBSnapshot, Value: ""}, - {Scope: ScopeSession, Name: TiDBOptAggPushDown, Value: BoolToIntStr(DefOptAggPushDown)}, + {Scope: ScopeSession, Name: TiDBOptAggPushDown, Value: BoolToIntStr(DefOptAggPushDown), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptBCJ, Value: BoolToIntStr(DefOptBCJ)}, - {Scope: ScopeSession, Name: TiDBOptDistinctAggPushDown, Value: BoolToIntStr(config.GetGlobalConfig().Performance.DistinctAggPushDown)}, + {Scope: ScopeSession, Name: TiDBOptDistinctAggPushDown, Value: BoolToIntStr(config.GetGlobalConfig().Performance.DistinctAggPushDown), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBOptWriteRowID, Value: BoolToIntStr(DefOptWriteRowID)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, Value: strconv.Itoa(DefBuildStatsConcurrency)}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeRatio, Value: strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64)}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime}, {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, Value: strconv.Itoa(DefChecksumTableConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDistSQLScanConcurrency, Value: strconv.Itoa(DefDistSQLScanConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptInSubqToJoinAndAgg, Value: BoolToIntStr(DefOptInSubqToJoinAndAgg)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBDistSQLScanConcurrency, Value: strconv.Itoa(DefDistSQLScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptInSubqToJoinAndAgg, Value: BoolToIntStr(DefOptInSubqToJoinAndAgg), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationThreshold, Value: strconv.FormatFloat(DefOptCorrelationThreshold, 'f', -1, 64)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationExpFactor, Value: strconv.Itoa(DefOptCorrelationExpFactor)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationExpFactor, Value: strconv.Itoa(DefOptCorrelationExpFactor), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCPUFactor, Value: strconv.FormatFloat(DefOptCPUFactor, 'f', -1, 64)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCopCPUFactor, Value: strconv.FormatFloat(DefOptCopCPUFactor, 'f', -1, 64)}, @@ -658,77 +667,77 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMemoryFactor, Value: strconv.FormatFloat(DefOptMemoryFactor, 'f', -1, 64)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDiskFactor, Value: strconv.FormatFloat(DefOptDiskFactor, 'f', -1, 64)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptConcurrencyFactor, Value: strconv.FormatFloat(DefOptConcurrencyFactor, 'f', -1, 64)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinBatchSize, Value: strconv.Itoa(DefIndexJoinBatchSize)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupSize, Value: strconv.Itoa(DefIndexLookupSize)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinBatchSize, Value: strconv.Itoa(DefIndexJoinBatchSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupSize, Value: strconv.Itoa(DefIndexLookupSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipUTF8Check, Value: BoolToIntStr(DefSkipUTF8Check)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipASCIICheck, Value: BoolToIntStr(DefSkipASCIICheck)}, - {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToIntStr(DefBatchInsert)}, - {Scope: ScopeSession, Name: TiDBBatchDelete, Value: BoolToIntStr(DefBatchDelete)}, - {Scope: ScopeSession, Name: TiDBBatchCommit, Value: BoolToIntStr(DefBatchCommit)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipUTF8Check, Value: BoolToIntStr(DefSkipUTF8Check), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipASCIICheck, Value: BoolToIntStr(DefSkipASCIICheck), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToIntStr(DefBatchInsert), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBBatchDelete, Value: BoolToIntStr(DefBatchDelete), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBBatchCommit, Value: BoolToIntStr(DefBatchCommit), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS)}, {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxChunkSize, Value: strconv.Itoa(DefMaxChunkSize)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowBatchCop, Value: strconv.Itoa(DefTiDBAllowBatchCop)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBInitChunkSize, Value: strconv.Itoa(DefInitChunkSize)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableCascadesPlanner, Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMerge, Value: "0"}, - {Scope: ScopeSession, Name: TIDBMemQuotaQuery, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaQuery, 10)}, - {Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10)}, - {Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10)}, - {Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10)}, - {Scope: ScopeSession, Name: TIDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10)}, - {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10)}, - {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10)}, - {Scope: ScopeSession, Name: TIDBMemQuotaNestedLoopApply, Value: strconv.FormatInt(DefTiDBMemQuotaNestedLoopApply, 10)}, - {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: "0"}, - {Scope: ScopeSession, Name: TiDBEnableChunkRPC, Value: "1"}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableCascadesPlanner, Value: "0", Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMerge, Value: "0", Type: TypeBool}, + {Scope: ScopeSession, Name: TIDBMemQuotaQuery, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaQuery, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TIDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TIDBMemQuotaNestedLoopApply, Value: strconv.FormatInt(DefTiDBMemQuotaNestedLoopApply, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: "0", Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBEnableChunkRPC, Value: "1", Type: TypeBool}, {Scope: ScopeSession, Name: TxnIsolationOneShot, Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: "on"}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashJoinConcurrency, Value: strconv.Itoa(DefTiDBHashJoinConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBProjectionConcurrency, Value: strconv.Itoa(DefTiDBProjectionConcurrency)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBProjectionConcurrency, Value: strconv.Itoa(DefTiDBProjectionConcurrency), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggPartialConcurrency, Value: strconv.Itoa(DefTiDBHashAggPartialConcurrency)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggFinalConcurrency, Value: strconv.Itoa(DefTiDBHashAggFinalConcurrency)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBWindowConcurrency, Value: strconv.Itoa(DefTiDBWindowConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableParallelApply, Value: BoolToIntStr(DefTiDBEnableParallelApply)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackoffLockFast, Value: strconv.Itoa(kv.DefBackoffLockFast)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackOffWeight, Value: strconv.Itoa(kv.DefBackOffWeight)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRetryLimit, Value: strconv.Itoa(DefTiDBRetryLimit)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDisableTxnAutoRetry, Value: BoolToIntStr(DefTiDBDisableTxnAutoRetry)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBConstraintCheckInPlace, Value: BoolToIntStr(DefTiDBConstraintCheckInPlace)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableParallelApply, Value: BoolToIntStr(DefTiDBEnableParallelApply), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackoffLockFast, Value: strconv.Itoa(kv.DefBackoffLockFast), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackOffWeight, Value: strconv.Itoa(kv.DefBackOffWeight), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBRetryLimit, Value: strconv.Itoa(DefTiDBRetryLimit), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBDisableTxnAutoRetry, Value: BoolToIntStr(DefTiDBDisableTxnAutoRetry), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBConstraintCheckInPlace, Value: BoolToIntStr(DefTiDBConstraintCheckInPlace), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBTxnMode, Value: DefTiDBTxnMode}, {Scope: ScopeGlobal, Name: TiDBRowFormatVersion, Value: strconv.Itoa(DefTiDBRowFormatV1)}, - {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableWindowFunction, Value: BoolToIntStr(DefEnableWindowFunction)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableVectorizedExpression, Value: BoolToIntStr(DefEnableVectorizedExpression)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableFastAnalyze, Value: BoolToIntStr(DefTiDBUseFastAnalyze)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipIsolationLevelCheck, Value: BoolToIntStr(DefTiDBSkipIsolationLevelCheck)}, + {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableWindowFunction, Value: BoolToIntStr(DefEnableWindowFunction), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableVectorizedExpression, Value: BoolToIntStr(DefEnableVectorizedExpression), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableFastAnalyze, Value: BoolToIntStr(DefTiDBUseFastAnalyze), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipIsolationLevelCheck, Value: BoolToIntStr(DefTiDBSkipIsolationLevelCheck), Type: TypeBool}, /* The following variable is defined as session scope but is actually server scope. */ - {Scope: ScopeSession, Name: TiDBGeneralLog, Value: strconv.Itoa(DefTiDBGeneralLog)}, - {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU)}, + {Scope: ScopeSession, Name: TiDBGeneralLog, Value: strconv.Itoa(DefTiDBGeneralLog), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBDDLSlowOprThreshold, Value: strconv.Itoa(DefTiDBDDLSlowOprThreshold)}, {Scope: ScopeSession, Name: TiDBConfig, Value: ""}, - {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount)}, - {Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize)}, - {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit)}, + {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true}, {Scope: ScopeSession, Name: TiDBDDLReorgPriority, Value: "PRIORITY_LOW"}, - {Scope: ScopeGlobal, Name: TiDBMaxDeltaSchemaCount, Value: strconv.Itoa(DefTiDBMaxDeltaSchemaCount)}, - {Scope: ScopeGlobal, Name: TiDBEnableChangeColumnType, Value: BoolToIntStr(DefTiDBChangeColumnType)}, + {Scope: ScopeGlobal, Name: TiDBMaxDeltaSchemaCount, Value: strconv.Itoa(DefTiDBMaxDeltaSchemaCount), Type: TypeUnsigned, MinValue: 100, MaxValue: 16384, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: TiDBEnableChangeColumnType, Value: BoolToIntStr(DefTiDBChangeColumnType), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBForcePriority, Value: mysql.Priority2Str[DefTiDBForcePriority]}, - {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToIntStr(DefTiDBUseRadixJoin)}, + {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToIntStr(DefTiDBUseRadixJoin), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold)}, {Scope: ScopeSession, Name: TiDBSlowQueryFile, Value: ""}, - {Scope: ScopeGlobal, Name: TiDBScatterRegion, Value: BoolToIntStr(DefTiDBScatterRegion)}, - {Scope: ScopeSession, Name: TiDBWaitSplitRegionFinish, Value: BoolToIntStr(DefTiDBWaitSplitRegionFinish)}, + {Scope: ScopeGlobal, Name: TiDBScatterRegion, Value: BoolToIntStr(DefTiDBScatterRegion), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBWaitSplitRegionFinish, Value: BoolToIntStr(DefTiDBWaitSplitRegionFinish), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(DefWaitSplitRegionTimeout)}, - {Scope: ScopeSession, Name: TiDBLowResolutionTSO, Value: "0"}, - {Scope: ScopeSession, Name: TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: BoolToIntStr(DefTiDBEnableNoopFuncs)}, + {Scope: ScopeSession, Name: TiDBLowResolutionTSO, Value: "0", Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold), Type: TypeUnsigned, MinValue: int64(MinExpensiveQueryTimeThreshold), MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: BoolToIntStr(DefTiDBEnableNoopFuncs), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBReplicaRead, Value: "leader"}, - {Scope: ScopeSession, Name: TiDBAllowRemoveAutoInc, Value: BoolToIntStr(DefTiDBAllowRemoveAutoInc)}, + {Scope: ScopeSession, Name: TiDBAllowRemoveAutoInc, Value: BoolToIntStr(DefTiDBAllowRemoveAutoInc), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableStmtSummary, Value: BoolToIntStr(config.GetGlobalConfig().StmtSummary.Enable)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryInternalQuery, Value: BoolToIntStr(config.GetGlobalConfig().StmtSummary.EnableInternalQuery)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.RefreshInterval)}, @@ -736,39 +745,39 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: "off"}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBUsePlanBaselines, Value: boolToOnOff(DefTiDBUsePlanBaselines)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEvolvePlanBaselines, Value: boolToOnOff(DefTiDBEvolvePlanBaselines)}, - {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskMaxTime, Value: strconv.Itoa(DefTiDBEvolvePlanTaskMaxTime)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBUsePlanBaselines, Value: boolToOnOff(DefTiDBUsePlanBaselines), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEvolvePlanBaselines, Value: boolToOnOff(DefTiDBEvolvePlanBaselines), Type: TypeBool}, + {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskMaxTime, Value: strconv.Itoa(DefTiDBEvolvePlanTaskMaxTime), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskStartTime, Value: DefTiDBEvolvePlanTaskStartTime}, {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskEndTime, Value: DefTiDBEvolvePlanTaskEndTime}, {Scope: ScopeSession, Name: TiDBIsolationReadEngines, Value: strings.Join(config.GetGlobalConfig().IsolationRead.Engines, ", ")}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStoreLimit, Value: strconv.FormatInt(atomic.LoadInt64(&config.GetGlobalConfig().TiKVClient.StoreLimit), 10)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStoreLimit, Value: strconv.FormatInt(atomic.LoadInt64(&config.GetGlobalConfig().TiKVClient.StoreLimit), 10), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true}, {Scope: ScopeSession, Name: TiDBMetricSchemaStep, Value: strconv.Itoa(DefTiDBMetricSchemaStep)}, {Scope: ScopeSession, Name: TiDBMetricSchemaRangeDuration, Value: strconv.Itoa(DefTiDBMetricSchemaRangeDuration)}, - {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold)}, - {Scope: ScopeSession, Name: TiDBRecordPlanInSlowLog, Value: strconv.Itoa(logutil.DefaultRecordPlanInSlowLog)}, - {Scope: ScopeSession, Name: TiDBEnableSlowLog, Value: BoolToIntStr(logutil.DefaultTiDBEnableSlowLog)}, - {Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen)}, - {Scope: ScopeSession, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, - {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToIntStr(DefTiDBFoundInPlanCache)}, - {Scope: ScopeSession, Name: TiDBEnableCollectExecutionInfo, Value: BoolToIntStr(DefTiDBEnableCollectExecutionInfo)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowAutoRandExplicitInsert, Value: boolToOnOff(DefTiDBAllowAutoRandExplicitInsert)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableClusteredIndex, Value: BoolToIntStr(DefTiDBEnableClusteredIndex)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: string(StaticOnly)}, + {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TiDBRecordPlanInSlowLog, Value: strconv.Itoa(logutil.DefaultRecordPlanInSlowLog), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBEnableSlowLog, Value: BoolToIntStr(logutil.DefaultTiDBEnableSlowLog), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: ScopeSession, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToIntStr(DefTiDBFoundInPlanCache), Type: TypeBool}, + {Scope: ScopeSession, Name: TiDBEnableCollectExecutionInfo, Value: BoolToIntStr(DefTiDBEnableCollectExecutionInfo), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowAutoRandExplicitInsert, Value: boolToOnOff(DefTiDBAllowAutoRandExplicitInsert), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableClusteredIndex, Value: BoolToIntStr(DefTiDBEnableClusteredIndex), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: string(StaticOnly), Type: TypeStr}, {Scope: ScopeGlobal, Name: TiDBSlowLogMasking, Value: BoolToIntStr(DefTiDBSlowLogMasking)}, {Scope: ScopeGlobal, Name: TiDBRedactLog, Value: strconv.Itoa(config.DefTiDBRedactLog)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBShardAllocateStep, Value: strconv.Itoa(DefTiDBShardAllocateStep)}, - {Scope: ScopeGlobal, Name: TiDBEnableTelemetry, Value: BoolToIntStr(DefTiDBEnableTelemetry)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAmendPessimisticTxn, Value: boolToOnOff(DefTiDBEnableAmendPessimisticTxn)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBShardAllocateStep, Value: strconv.Itoa(DefTiDBShardAllocateStep), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: TiDBEnableTelemetry, Value: BoolToIntStr(DefTiDBEnableTelemetry), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAmendPessimisticTxn, Value: boolToOnOff(DefTiDBEnableAmendPessimisticTxn), Type: TypeBool}, // for compatibility purpose, we should leave them alone. // TODO: Follow the Terminology Updates of MySQL after their changes arrived. // https://mysqlhighavailability.com/mysql-terminology-updates/ - {Scope: ScopeSession, Name: PseudoSlaveMode, Value: ""}, + {Scope: ScopeSession, Name: PseudoSlaveMode, Value: "", Type: TypeBool}, {Scope: ScopeGlobal, Name: "slave_pending_jobs_size_max", Value: "16777216"}, {Scope: ScopeGlobal, Name: "slave_transaction_retries", Value: "10"}, {Scope: ScopeGlobal, Name: "slave_checkpoint_period", Value: "300"}, - {Scope: ScopeGlobal, Name: MasterVerifyChecksum, Value: "0"}, + {Scope: ScopeGlobal, Name: MasterVerifyChecksum, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_trace_level", Value: ""}, {Scope: ScopeGlobal, Name: "master_info_repository", Value: "FILE"}, {Scope: ScopeGlobal, Name: "rpl_stop_slave_timeout", Value: "31536000"}, @@ -777,7 +786,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "init_slave", Value: ""}, {Scope: ScopeGlobal, Name: SlaveCompressedProtocol, Value: "0"}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_slave_trace_level", Value: ""}, - {Scope: ScopeGlobal, Name: LogSlowSlaveStatements, Value: "0"}, + {Scope: ScopeGlobal, Name: LogSlowSlaveStatements, Value: "0", Type: TypeBool}, {Scope: ScopeGlobal, Name: "slave_checkpoint_group", Value: "512"}, {Scope: ScopeNone, Name: "slave_load_tmpdir", Value: "/var/tmp/"}, {Scope: ScopeGlobal, Name: "slave_parallel_type", Value: ""}, @@ -795,8 +804,8 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "slave_max_allowed_packet", Value: "1073741824"}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_for_slave_count", Value: ""}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_no_slave", Value: ""}, - {Scope: ScopeGlobal, Name: "slave_rows_search_algorithms", Value: "TABLE_SCANINDEX_SCAN"}, - {Scope: ScopeGlobal, Name: SlaveAllowBatching, Value: "0"}, + {Scope: ScopeGlobal, Name: "slave_rows_search_algorithms", Value: "TABLE_SCAN,INDEX_SCAN"}, + {Scope: ScopeGlobal, Name: SlaveAllowBatching, Value: "0", Type: TypeBool}, } // SynonymsSysVariables is synonyms of system variables. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index d05fc43b7dd82..a3f890e7ef017 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -552,6 +552,7 @@ const ( DefTiDBEnableTelemetry = true DefTiDBEnableParallelApply = false DefTiDBEnableAmendPessimisticTxn = true + DefTiDBPartitionPruneMode = "static-only" ) // Process global variables. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index cce4606b9c49e..3bf9264e531ee 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -209,7 +209,7 @@ const epochShiftBits = 18 func SetSessionSystemVar(vars *SessionVars, name string, value types.Datum) error { sysVar := GetSysVar(name) if sysVar == nil { - return ErrUnknownSystemVar + return ErrUnknownSystemVar.GenWithStackByArgs(name) } sVal := "" var err error @@ -247,6 +247,13 @@ func ValidateGetSystemVar(name string, isGlobal bool) error { } func checkUInt64SystemVar(name, value string, min, max uint64, vars *SessionVars) (string, error) { + // There are two types of validation behaviors for integer values. The default + // is to return an error saying the value is out of range. For MySQL compatibility, some + // values prefer convert the value to the min/max and return a warning. + sv := GetSysVar(name) + if sv != nil && !sv.AutoConvertOutOfRange { + return checkUint64SystemVarWithError(name, value, min, max) + } if len(value) == 0 { return value, ErrWrongTypeForVar.GenWithStackByArgs(name) } @@ -274,6 +281,13 @@ func checkUInt64SystemVar(name, value string, min, max uint64, vars *SessionVars } func checkInt64SystemVar(name, value string, min, max int64, vars *SessionVars) (string, error) { + // There are two types of validation behaviors for integer values. The default + // is to return an error saying the value is out of range. For MySQL compatibility, some + // values prefer convert the value to the min/max and return a warning. + sv := GetSysVar(name) + if sv != nil && !sv.AutoConvertOutOfRange { + return checkInt64SystemVarWithError(name, value, min, max) + } val, err := strconv.ParseInt(value, 10, 64) if err != nil { return value, ErrWrongTypeForVar.GenWithStackByArgs(name) @@ -289,7 +303,57 @@ func checkInt64SystemVar(name, value string, min, max int64, vars *SessionVars) return value, nil } +func checkBoolSystemVar(name, value string, vars *SessionVars) (string, error) { + if strings.EqualFold(value, "ON") { + return "1", nil + } else if strings.EqualFold(value, "OFF") { + return "0", nil + } + val, err := strconv.ParseInt(value, 10, 64) + if err == nil { + // Confusingly, there are two types of conversion rules for integer values. + // The default only allows 0 || 1, but a subset of values convert any + // negative integer to 1. + sv := GetSysVar(name) + if !sv.AutoConvertNegativeBool { + if val == 0 { + return "0", nil + } else if val == 1 { + return "1", nil + } + } else { + if val == 1 || val < 0 { + return "1", nil + } else if val == 0 { + return "0", nil + } + } + } + return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) +} + +func checkUint64SystemVarWithError(name, value string, min, max uint64) (string, error) { + if len(value) == 0 { + return value, ErrWrongTypeForVar.GenWithStackByArgs(name) + } + if value[0] == '-' { + // // in strict it expects the error WrongValue, but in non-strict it returns WrongType + return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) + } + val, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(name) + } + if val < min || val > max { + return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) + } + return value, nil +} + func checkInt64SystemVarWithError(name, value string, min, max int64) (string, error) { + if len(value) == 0 { + return value, ErrWrongTypeForVar.GenWithStackByArgs(name) + } val, err := strconv.ParseInt(value, 10, 64) if err != nil { return value, ErrWrongTypeForVar.GenWithStackByArgs(name) @@ -324,17 +388,34 @@ func CheckDeprecationSetSystemVar(s *SessionVars, name string) { // ValidateSetSystemVar checks if system variable satisfies specific restriction. func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope ScopeFlag) (string, error) { + sv := GetSysVar(name) + // The string "DEFAULT" is a special keyword in MySQL, which restores + // the compiled sysvar value. In which case we can skip further validation. if strings.EqualFold(value, "DEFAULT") { - if val := GetSysVar(name); val != nil { - return val.Value, nil + if sv != nil { + return sv.Value, nil } return value, ErrUnknownSystemVar.GenWithStackByArgs(name) } + // Attempt to provide validation using the SysVar struct. + // Eventually the struct should handle all validation + var err error + if sv != nil { + switch sv.Type { + case TypeUnsigned: + value, err = checkUInt64SystemVar(name, value, uint64(sv.MinValue), sv.MaxValue, vars) + case TypeInt: + value, err = checkInt64SystemVar(name, value, sv.MinValue, int64(sv.MaxValue), vars) + case TypeBool: + value, err = checkBoolSystemVar(name, value, vars) + } + // If there is no error, follow through and handle legacy cases of validation that are not handled by the type. + // TODO: Move each of these validations into the SysVar as an anonymous function. + if err != nil { + return value, err + } + } switch name { - case ConnectTimeout: - return checkUInt64SystemVar(name, value, 2, secondsPerYear, vars) - case DefaultWeekFormat: - return checkUInt64SystemVar(name, value, 0, 7, vars) case DelayKeyWrite: if strings.EqualFold(value, "ON") || value == "1" { return "ON", nil @@ -344,8 +425,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return "ALL", nil } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - case FlushTime: - return checkUInt64SystemVar(name, value, 0, secondsPerYear, vars) case ForeignKeyChecks: if strings.EqualFold(value, "ON") || value == "1" { // TiDB does not yet support foreign keys. @@ -366,31 +445,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc maxLen = uint64(math.MaxUint32) } return checkUInt64SystemVar(name, value, 4, maxLen, vars) - case InteractiveTimeout: - return checkUInt64SystemVar(name, value, 1, secondsPerYear, vars) - case InnodbCommitConcurrency: - return checkUInt64SystemVar(name, value, 0, 1000, vars) - case InnodbFastShutdown: - return checkUInt64SystemVar(name, value, 0, 2, vars) - case InnodbLockWaitTimeout: - return checkUInt64SystemVar(name, value, 1, 1073741824, vars) - // See "https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_max_allowed_packet" - case MaxAllowedPacket: - return checkUInt64SystemVar(name, value, 1024, MaxOfMaxAllowedPacket, vars) - case MaxConnections: - return checkUInt64SystemVar(name, value, 1, 100000, vars) - case MaxConnectErrors: - return checkUInt64SystemVar(name, value, 1, math.MaxUint64, vars) - case MaxSortLength: - return checkUInt64SystemVar(name, value, 4, 8388608, vars) - case MaxSpRecursionDepth: - return checkUInt64SystemVar(name, value, 0, 255, vars) - case MaxUserConnections: - return checkUInt64SystemVar(name, value, 0, 4294967295, vars) - case OldPasswords: - return checkUInt64SystemVar(name, value, 0, 2, vars) - case TiDBMaxDeltaSchemaCount: - return checkInt64SystemVar(name, value, 100, 16384, vars) case SessionTrackGtids: if strings.EqualFold(value, "OFF") || value == "0" { return "OFF", nil @@ -400,30 +454,12 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return "ALL_GTIDS", nil } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - case SQLSelectLimit: - return checkUInt64SystemVar(name, value, 0, math.MaxUint64, vars) - case TiDBStoreLimit: - return checkInt64SystemVar(name, value, 0, math.MaxInt64, vars) - case SyncBinlog: - return checkUInt64SystemVar(name, value, 0, 4294967295, vars) - case TableDefinitionCache: - return checkUInt64SystemVar(name, value, 400, 524288, vars) - case TmpTableSize: - return checkUInt64SystemVar(name, value, 1024, math.MaxUint64, vars) - case WaitTimeout: - return checkUInt64SystemVar(name, value, 0, 31536000, vars) - case MaxPreparedStmtCount: - return checkInt64SystemVar(name, value, -1, 1048576, vars) - case AutoIncrementIncrement, AutoIncrementOffset: - return checkUInt64SystemVar(name, value, 1, math.MaxUint16, vars) case TimeZone: if strings.EqualFold(value, "SYSTEM") { return "SYSTEM", nil } _, err := parseTimeZone(value) return value, err - case ValidatePasswordLength, ValidatePasswordNumberCount: - return checkUInt64SystemVar(name, value, 0, math.MaxUint64, vars) case WarningCount, ErrorCount: return value, ErrReadOnly.GenWithStackByArgs(name) case EnforceGtidConsistency: @@ -461,61 +497,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return value, ErrWrongValueForVar.GenWithStackByArgs("Can't set Broadcast Join to 1 but tidb_allow_batch_cop is 0, please active batch cop at first.") } return value, nil - case TiDBSkipUTF8Check, TiDBSkipASCIICheck, TiDBOptAggPushDown, - TiDBOptDistinctAggPushDown, TiDBOptInSubqToJoinAndAgg, TiDBEnableFastAnalyze, - TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming, TiDBEnableChunkRPC, - TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, TiDBPProfSQLCPU, - TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs, - TiDBCheckMb4ValueInUTF8, TiDBEnableSlowLog, TiDBRecordPlanInSlowLog, - TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace, TiDBEnableVectorizedExpression, - TiDBFoundInPlanCache, TiDBEnableCollectExecutionInfo, TiDBAllowAutoRandExplicitInsert, - TiDBEnableClusteredIndex, TiDBEnableTelemetry, TiDBEnableChangeColumnType, TiDBEnableAmendPessimisticTxn: - fallthrough - case GeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin, - CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates, - SkipNameResolve, SQLSafeUpdates, serverReadOnly, SlaveAllowBatching, - Flush, PerformanceSchema, LocalInFile, ShowOldTemporals, KeepFilesOnCreate, AutoCommit, - SQLWarnings, UniqueChecks, OldAlterTable, LogBinTrustFunctionCreators, SQLBigSelects, - BinlogDirectNonTransactionalUpdates, SQLQuoteShowCreate, AutomaticSpPrivileges, - RelayLogPurge, SQLAutoIsNull, QueryCacheWlockInvalidate, ValidatePasswordCheckUserName, - SuperReadOnly, BinlogOrderCommits, MasterVerifyChecksum, BinlogRowQueryLogEvents, LogSlowSlaveStatements, - LogSlowAdminStatements, LogQueriesNotUsingIndexes, Profiling: - if strings.EqualFold(value, "ON") { - return "1", nil - } else if strings.EqualFold(value, "OFF") { - return "0", nil - } - val, err := strconv.ParseInt(value, 10, 64) - if err == nil { - if val == 0 { - return "0", nil - } else if val == 1 { - return "1", nil - } - } - return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - case MyISAMUseMmap, InnodbTableLocks, InnodbStatusOutput, InnodbAdaptiveFlushing, InnodbRandomReadAhead, - InnodbStatsPersistent, InnodbBufferPoolLoadAbort, InnodbBufferPoolLoadNow, InnodbBufferPoolDumpNow, - InnodbCmpPerIndexEnabled, InnodbFilePerTable, InnodbPrintAllDeadlocks, - InnodbStrictMode, InnodbAdaptiveHashIndex, InnodbFtEnableStopword, InnodbStatusOutputLocks: - if strings.EqualFold(value, "ON") { - return "1", nil - } else if strings.EqualFold(value, "OFF") { - return "0", nil - } - val, err := strconv.ParseInt(value, 10, 64) - if err == nil { - if val == 1 || val < 0 { - return "1", nil - } else if val == 0 { - return "0", nil - } - } - return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - case MaxExecutionTime: - return checkUInt64SystemVar(name, value, 0, math.MaxUint64, vars) - case ThreadPoolSize: - return checkUInt64SystemVar(name, value, 1, 64, vars) case TiDBEnableTablePartition: switch { case strings.EqualFold(value, "ON") || value == "1": @@ -526,12 +507,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return "auto", nil } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - case TiDBDDLReorgBatchSize: - return checkUInt64SystemVar(name, value, uint64(MinDDLReorgBatchSize), uint64(MaxDDLReorgBatchSize), vars) - case TiDBDDLErrorCountLimit: - return checkUInt64SystemVar(name, value, uint64(0), math.MaxInt64, vars) - case TiDBExpensiveQueryTimeThreshold: - return checkUInt64SystemVar(name, value, MinExpensiveQueryTimeThreshold, math.MaxInt64, vars) case TiDBIndexLookupConcurrency, TiDBIndexLookupJoinConcurrency, TiDBHashJoinConcurrency, @@ -546,31 +521,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) } return value, nil - case TiDBExecutorConcurrency, - TiDBDistSQLScanConcurrency, - TiDBIndexSerialScanConcurrency, - TiDBIndexJoinBatchSize, - TiDBIndexLookupSize, - TiDBDDLReorgWorkerCount, - TiDBBackoffLockFast, TiDBBackOffWeight, - TiDBOptimizerSelectivityLevel: - v, err := strconv.Atoi(value) - if err != nil { - return value, ErrWrongTypeForVar.GenWithStackByArgs(name) - } - if v <= 0 { - return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - } - return value, nil - case TiDBOptCorrelationExpFactor, TiDBDMLBatchSize: - v, err := strconv.Atoi(value) - if err != nil { - return value, ErrWrongTypeForVar.GenWithStackByArgs(name) - } - if v < 0 { - return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - } - return value, nil case TiDBOptCorrelationThreshold: v, err := strconv.ParseFloat(value, 64) if err != nil { @@ -610,24 +560,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) } return value, nil - case TiDBProjectionConcurrency, - TIDBMemQuotaQuery, - TIDBMemQuotaHashJoin, - TIDBMemQuotaMergeJoin, - TIDBMemQuotaSort, - TIDBMemQuotaTopn, - TIDBMemQuotaIndexLookupReader, - TIDBMemQuotaIndexLookupJoin, - TIDBMemQuotaNestedLoopApply, - TiDBRetryLimit, - TiDBSlowLogThreshold, - TiDBQueryLogMaxLen, - TiDBEvolvePlanTaskMaxTime: - _, err := strconv.ParseInt(value, 10, 64) - if err != nil { - return value, ErrWrongValueForVar.GenWithStackByArgs(name) - } - return value, nil case TiDBAutoAnalyzeStartTime, TiDBAutoAnalyzeEndTime, TiDBEvolvePlanTaskStartTime, TiDBEvolvePlanTaskEndTime: v, err := setDayTime(vars, value) if err != nil { @@ -812,8 +744,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc if _, err := collate.GetCollationByName(value); err != nil { return value, errors.Trace(err) } - case TiDBShardAllocateStep: - return checkInt64SystemVar(name, value, 1, math.MaxInt64, vars) } return value, nil } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 9dbf0443f3a4e..946d12bd86cb8 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -466,6 +466,9 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(err, IsNil) c.Assert(val, Equals, "1") c.Assert(v.systems[TiDBEnableChangeColumnType], Equals, "1") + + err = SetSessionSystemVar(v, "UnknownVariable", types.NewStringDatum("on")) + c.Assert(err, ErrorMatches, ".*]Unknown system variable 'UnknownVariable'") } func (s *testVarsutilSuite) TestSetOverflowBehave(c *C) { diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 9881ee3b52d94..204f387b9e539 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/sqlexec" ) @@ -30,37 +31,54 @@ import ( func (h *Handle) HandleDDLEvent(t *util.Event) error { switch t.Tp { case model.ActionCreateTable, model.ActionTruncateTable: - ids := getPhysicalIDs(t.TableInfo) + ids := h.getInitStateTableIDs(t.TableInfo) for _, id := range ids { if err := h.insertTableStats2KV(t.TableInfo, id); err != nil { return err } } case model.ActionAddColumn, model.ActionAddColumns, model.ActionModifyColumn: - ids := getPhysicalIDs(t.TableInfo) + ids := h.getInitStateTableIDs(t.TableInfo) for _, id := range ids { if err := h.insertColStats2KV(id, t.ColumnInfos); err != nil { return err } } case model.ActionAddTablePartition, model.ActionTruncateTablePartition: - for _, def := range t.PartInfo.Definitions { - if err := h.insertTableStats2KV(t.TableInfo, def.ID); err != nil { - return err + pruneMode := h.CurrentPruneMode() + if pruneMode == variable.StaticOnly || pruneMode == variable.StaticButPrepareDynamic { + for _, def := range t.PartInfo.Definitions { + if err := h.insertTableStats2KV(t.TableInfo, def.ID); err != nil { + return err + } } } + if pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { + // TODO: need trigger full analyze + } + case model.ActionDropTablePartition: + pruneMode := h.CurrentPruneMode() + if pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { + // TODO: need trigger full analyze + } } return nil } -func getPhysicalIDs(tblInfo *model.TableInfo) []int64 { +func (h *Handle) getInitStateTableIDs(tblInfo *model.TableInfo) (ids []int64) { pi := tblInfo.GetPartitionInfo() if pi == nil { return []int64{tblInfo.ID} } - ids := make([]int64, 0, len(pi.Definitions)) - for _, def := range pi.Definitions { - ids = append(ids, def.ID) + ids = make([]int64, 0, len(pi.Definitions)+1) + pruneMode := h.CurrentPruneMode() + if pruneMode == variable.StaticOnly || pruneMode == variable.StaticButPrepareDynamic { + for _, def := range pi.Definitions { + ids = append(ids, def.ID) + } + } + if pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { + ids = append(ids, tblInfo.ID) } return ids } @@ -143,7 +161,7 @@ func (h *Handle) insertColStats2KV(physicalID int64, colInfos []*model.ColumnInf count := req.GetRow(0).GetInt64(0) sqls := make([]string, 0, len(colInfos)) for _, colInfo := range colInfos { - value := types.NewDatum(colInfo.OriginDefaultValue) + value := types.NewDatum(colInfo.GetOriginDefaultValue()) value, err = value.ConvertTo(h.mu.ctx.GetSessionVars().StmtCtx, &colInfo.FieldType) if err != nil { return diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 18729063b48a9..1ef91f1f4698e 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/sqlexec" @@ -114,7 +115,7 @@ func dumpJSONCol(hist *statistics.Histogram, CMSketch *statistics.CMSketch) *jso // DumpStatsToJSON dumps statistic to json. func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo, historyStatsExec sqlexec.RestrictedSQLExecutor) (*JSONTable, error) { pi := tableInfo.GetPartitionInfo() - if pi == nil { + if pi == nil || h.CurrentPruneMode() == variable.DynamicOnly { return h.tableStatsToJSON(dbName, tableInfo, tableInfo.ID, historyStatsExec) } jsonTbl := &JSONTable{ @@ -177,15 +178,12 @@ func (h *Handle) LoadStatsFromJSON(is infoschema.InfoSchema, jsonTbl *JSONTable) } tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() - if pi == nil { + if pi == nil || jsonTbl.Partitions == nil { err := h.loadStatsFromJSON(tableInfo, tableInfo.ID, jsonTbl) if err != nil { return errors.Trace(err) } } else { - if jsonTbl.Partitions == nil { - return errors.New("No partition statistics") - } for _, def := range pi.Definitions { tbl := jsonTbl.Partitions[def.Name.L] if tbl == nil { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 2eaa839037e17..1322068479a11 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -112,7 +112,7 @@ func (h *Handle) Clear() { } // NewHandle creates a Handle for update stats. -func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { +func NewHandle(ctx sessionctx.Context, lease time.Duration) (*Handle, error) { handle := &Handle{ ddlEventCh: make(chan *util.Event, 100), listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, @@ -128,7 +128,11 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { handle.mu.ctx = ctx handle.mu.rateMap = make(errorRateDeltaMap) handle.statsCache.Store(statsCache{tables: make(map[int64]*statistics.Table)}) - return handle + err := handle.RefreshVars() + if err != nil { + return nil, err + } + return handle, nil } // Lease returns the stats lease. @@ -195,7 +199,7 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, false, nil) // Error is not nil may mean that there are some ddl changes on this table, we will not update it. if err != nil { - logutil.BgLogger().Debug("error occurred when read table stats", zap.String("table", tableInfo.Name.O), zap.Error(err)) + logutil.BgLogger().Error("[stats] error occurred when read table stats", zap.String("table", tableInfo.Name.O), zap.Error(err)) continue } if tbl == nil { @@ -393,14 +397,14 @@ func (h *Handle) FlushStats() { for len(h.ddlEventCh) > 0 { e := <-h.ddlEventCh if err := h.HandleDDLEvent(e); err != nil { - logutil.BgLogger().Debug("[stats] handle ddl event fail", zap.Error(err)) + logutil.BgLogger().Error("[stats] handle ddl event fail", zap.Error(err)) } } if err := h.DumpStatsDeltaToKV(DumpAll); err != nil { - logutil.BgLogger().Debug("[stats] dump stats delta fail", zap.Error(err)) + logutil.BgLogger().Error("[stats] dump stats delta fail", zap.Error(err)) } if err := h.DumpStatsFeedbackToKV(); err != nil { - logutil.BgLogger().Debug("[stats] dump stats feedback fail", zap.Error(err)) + logutil.BgLogger().Error("[stats] dump stats feedback fail", zap.Error(err)) } } @@ -625,7 +629,7 @@ func (h *Handle) extendedStatsFromStorage(reader *statsReader, table *statistics colIDs := row.GetString(4) err := json.Unmarshal([]byte(colIDs), &item.ColIDs) if err != nil { - logutil.BgLogger().Debug("decode column IDs failed", zap.String("column_ids", colIDs), zap.Error(err)) + logutil.BgLogger().Error("[stats] decode column IDs failed", zap.String("column_ids", colIDs), zap.Error(err)) return nil, err } table.ExtendedStats.Stats[key] = item @@ -833,7 +837,7 @@ func (sr *statsReader) isHistory() bool { return sr.history != nil } -func (h *Handle) getStatsReader(history sqlexec.RestrictedSQLExecutor) (*statsReader, error) { +func (h *Handle) getStatsReader(history sqlexec.RestrictedSQLExecutor) (reader *statsReader, err error) { failpoint.Inject("mockGetStatsReaderFail", func(val failpoint.Value) { if val.(bool) { failpoint.Return(nil, errors.New("gofail genStatsReader error")) @@ -843,7 +847,16 @@ func (h *Handle) getStatsReader(history sqlexec.RestrictedSQLExecutor) (*statsRe return &statsReader{history: history}, nil } h.mu.Lock() - _, err := h.mu.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "begin") + defer func() { + if r := recover(); r != nil { + err = fmt.Errorf("getStatsReader panic %v", r) + } + if err != nil { + h.mu.Unlock() + } + }() + failpoint.Inject("mockGetStatsReaderPanic", nil) + _, err = h.mu.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "begin") if err != nil { return nil, err } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 3cc530b47cc70..e5c3d6e573182 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -354,7 +354,8 @@ func (s *testStatsSuite) TestVersion(c *C) { tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) tableInfo1 := tbl1.Meta() - h := handle.NewHandle(testKit.Se, time.Millisecond) + h, err := handle.NewHandle(testKit.Se, time.Millisecond) + c.Assert(err, IsNil) unit := oracle.ComposeTS(1, 0) testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", 2*unit, tableInfo1.ID) @@ -550,6 +551,13 @@ func (s *testStatsSuite) TestLoadStats(c *C) { err = h.LoadNeededHistograms() c.Assert(err, NotNil) c.Assert(failpoint.Disable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderFail"), IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderPanic", "panic"), IsNil) + err = h.LoadNeededHistograms() + c.Assert(err, ErrorMatches, ".*getStatsReader panic.*") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderPanic"), IsNil) + err = h.LoadNeededHistograms() + c.Assert(err, IsNil) } func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 28f331761f9fa..38f9b3c70aae2 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -581,10 +581,10 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch return nil } var tbl *statistics.Table - if table.Meta().GetPartitionInfo() != nil { - tbl = h.GetPartitionStats(table.Meta(), physicalTableID) - } else { + if table.Meta().GetPartitionInfo() == nil || h.CurrentPruneMode() == variable.DynamicOnly { tbl = h.GetTableStats(table.Meta()) + } else { + tbl = h.GetPartitionStats(table.Meta(), physicalTableID) } var cms *statistics.CMSketch var hist *statistics.Histogram @@ -736,12 +736,12 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) { logutil.BgLogger().Error("[stats] parse auto analyze period failed", zap.Error(err)) return } + pruneMode := h.CurrentPruneMode() for _, db := range dbs { tbls := is.SchemaTables(model.NewCIStr(db)) for _, tbl := range tbls { tblInfo := tbl.Meta() pi := tblInfo.GetPartitionInfo() - pruneMode := h.CurrentPruneMode() if pi == nil || pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { statsTbl := h.GetTableStats(tblInfo) sql := "analyze table `" + db + "`.`" + tblInfo.Name.O + "`" @@ -751,7 +751,7 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) { } continue } - if h.CurrentPruneMode() == variable.StaticOnly || pruneMode == variable.StaticButPrepareDynamic { + if pruneMode == variable.StaticOnly || pruneMode == variable.StaticButPrepareDynamic { for _, def := range pi.Definitions { sql := "analyze table `" + db + "`.`" + tblInfo.Name.O + "`" + " partition `" + def.Name.O + "`" statsTbl := h.GetPartitionStats(tblInfo, def.ID) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 25c9b933c7854..4c41057699a65 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -321,51 +321,55 @@ func (s *testStatsSuite) TestTxnWithFailure(c *C) { func (s *testStatsSuite) TestUpdatePartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) + testKit.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows(string(s.do.StatsHandle().CurrentPruneMode()))) testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - createTable := `CREATE TABLE t (a int, b char(5)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11))` - testKit.MustExec(createTable) - do := s.do - is := do.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tableInfo := tbl.Meta() - h := do.StatsHandle() - err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - pi := tableInfo.GetPartitionInfo() - c.Assert(len(pi.Definitions), Equals, 2) - bColID := tableInfo.Columns[1].ID + testkit.WithPruneMode(testKit, variable.StaticOnly, func() { + s.do.StatsHandle().RefreshVars() + testKit.MustExec("drop table if exists t") + createTable := `CREATE TABLE t (a int, b char(5)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11))` + testKit.MustExec(createTable) + do := s.do + is := do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + h := do.StatsHandle() + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) + pi := tableInfo.GetPartitionInfo() + c.Assert(len(pi.Definitions), Equals, 2) + bColID := tableInfo.Columns[1].ID - testKit.MustExec(`insert into t values (1, "a"), (7, "a")`) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) - for _, def := range pi.Definitions { - statsTbl := h.GetPartitionStats(tableInfo, def.ID) - c.Assert(statsTbl.ModifyCount, Equals, int64(1)) - c.Assert(statsTbl.Count, Equals, int64(1)) - c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(2)) - } + testKit.MustExec(`insert into t values (1, "a"), (7, "a")`) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + c.Assert(h.Update(is), IsNil) + for _, def := range pi.Definitions { + statsTbl := h.GetPartitionStats(tableInfo, def.ID) + c.Assert(statsTbl.ModifyCount, Equals, int64(1)) + c.Assert(statsTbl.Count, Equals, int64(1)) + c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(2)) + } - testKit.MustExec(`update t set a = a + 1, b = "aa"`) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) - for _, def := range pi.Definitions { - statsTbl := h.GetPartitionStats(tableInfo, def.ID) - c.Assert(statsTbl.ModifyCount, Equals, int64(2)) - c.Assert(statsTbl.Count, Equals, int64(1)) - c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(3)) - } + testKit.MustExec(`update t set a = a + 1, b = "aa"`) + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + c.Assert(h.Update(is), IsNil) + for _, def := range pi.Definitions { + statsTbl := h.GetPartitionStats(tableInfo, def.ID) + c.Assert(statsTbl.ModifyCount, Equals, int64(2)) + c.Assert(statsTbl.Count, Equals, int64(1)) + c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(3)) + } - testKit.MustExec("delete from t") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) - for _, def := range pi.Definitions { - statsTbl := h.GetPartitionStats(tableInfo, def.ID) - c.Assert(statsTbl.ModifyCount, Equals, int64(3)) - c.Assert(statsTbl.Count, Equals, int64(0)) - c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(0)) - } + testKit.MustExec("delete from t") + c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + c.Assert(h.Update(is), IsNil) + for _, def := range pi.Definitions { + statsTbl := h.GetPartitionStats(tableInfo, def.ID) + c.Assert(statsTbl.ModifyCount, Equals, int64(3)) + c.Assert(statsTbl.Count, Equals, int64(0)) + c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(0)) + } + }) } func (s *testStatsSuite) TestAutoUpdate(c *C) { diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 9d2ebc089e7ca..761deb2592b59 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/store/tikv/oracle" @@ -959,29 +960,45 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { logutil.SetTag(ctx, "commitTs", commitTS) } - tryAmend := c.isPessimistic && c.connID > 0 && !c.isAsyncCommit() && c.txn.schemaAmender != nil - if !tryAmend { - _, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, false) + if c.connID > 0 { + failpoint.Inject("beforeSchemaCheck", func() { + failpoint.Return() + }) + } + + if c.isAsyncCommit() { + schemaVerIsTheSame, err := checkSchemaVersionForAsyncCommit(ctx, c.startTS, commitTS, c.store) if err != nil { return errors.Trace(err) } - } else { - relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, true) - if err != nil { - return errors.Trace(err) + if !schemaVerIsTheSame { + return errors.Trace(errors.Errorf("Schema changed for async commit startTS=%v commitTS=%v", c.startTS, commitTS)) } - if memAmended { - // Get new commitTS and check schema valid again. - newCommitTS, err := c.getCommitTS(ctx, commitDetail) + } else { + tryAmend := c.isPessimistic && c.connID > 0 && c.txn.schemaAmender != nil + if !tryAmend { + _, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, false) if err != nil { return errors.Trace(err) } - // If schema check failed between commitTS and newCommitTs, report schema change error. - _, _, err = c.checkSchemaValid(ctx, newCommitTS, relatedSchemaChange.LatestInfoSchema, false) + } else { + relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, true) if err != nil { return errors.Trace(err) } - commitTS = newCommitTS + if memAmended { + // Get new commitTS and check schema valid again. + newCommitTS, err := c.getCommitTS(ctx, commitDetail) + if err != nil { + return errors.Trace(err) + } + // If schema check failed between commitTS and newCommitTs, report schema change error. + _, _, err = c.checkSchemaValid(ctx, newCommitTS, relatedSchemaChange.LatestInfoSchema, false) + if err != nil { + return errors.Trace(err) + } + commitTS = newCommitTS + } } } c.commitTS = commitTS @@ -999,8 +1016,9 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { if c.isAsyncCommit() { // For async commit protocol, the commit is considered success here. c.txn.commitTS = c.commitTS - logutil.Logger(ctx).Info("2PC will use async commit protocol to commit this txn", zap.Uint64("startTS", c.startTS), - zap.Uint64("commitTS", c.commitTS)) + logutil.Logger(ctx).Info("2PC will use async commit protocol to commit this txn", + zap.Uint64("startTS", c.startTS), zap.Uint64("commitTS", c.commitTS), + zap.Uint64("connID", c.connID)) go func() { failpoint.Inject("asyncCommitDoNothing", func() { failpoint.Return() @@ -1476,3 +1494,41 @@ func (c *twoPhaseCommitter) getUndeterminedErr() error { defer c.mu.RUnlock() return c.mu.undeterminedErr } + +// checkSchemaVersionForAsyncCommit is used to check schema version change for async commit transactions +// only. For async commit protocol, we need to make sure the check result is the same during common execution +// path and the recovery path. As the schema lease checker has a limited size of cached schema diff version, it's +// possible the schema cache is changed and the schema lease checker can't decide if the related table has +// schema version change. So we just check the version from meta snapshot, it's much stricter. +func checkSchemaVersionForAsyncCommit(ctx context.Context, startTS uint64, commitTS uint64, store Storage) (bool, error) { + if commitTS > 0 { + snapshotAtStart, err := store.GetSnapshot(kv.NewVersion(startTS)) + if err != nil { + logutil.Logger(ctx).Error("get snapshot failed for resolve async startTS", + zap.Uint64("startTS", startTS), zap.Uint64("commitTS", commitTS)) + return false, errors.Trace(err) + } + snapShotAtCommit, err := store.GetSnapshot(kv.NewVersion(commitTS)) + if err != nil { + logutil.Logger(ctx).Error("get snapshot failed for resolve async commitTS", + zap.Uint64("startTS", startTS), zap.Uint64("commitTS", commitTS)) + return false, errors.Trace(err) + } + schemaVerAtStart, err := meta.NewSnapshotMeta(snapshotAtStart).GetSchemaVersion() + if err != nil { + return false, errors.Trace(err) + } + schemaVerAtCommit, err := meta.NewSnapshotMeta(snapShotAtCommit).GetSchemaVersion() + if err != nil { + return false, errors.Trace(err) + } + if schemaVerAtStart != schemaVerAtCommit { + logutil.Logger(ctx).Info("async commit txn need to rollback since schema version has changed", + zap.Uint64("startTS", startTS), zap.Uint64("commitTS", commitTS), + zap.Int64("schema version at start", schemaVerAtStart), + zap.Int64("schema version at commit", schemaVerAtCommit)) + return false, nil + } + } + return true, nil +} diff --git a/store/tikv/2pc_test.go b/store/tikv/2pc_test.go index 8bbb53225a0f2..27cce10ed2ced 100644 --- a/store/tikv/2pc_test.go +++ b/store/tikv/2pc_test.go @@ -1228,8 +1228,6 @@ func (s *testCommitterSuite) TestAsyncCommit(c *C) { err = committer.execute(ctx) c.Assert(err, IsNil) - // TODO remove sleep when recovery logic is done - time.Sleep(1 * time.Second) s.checkValues(c, map[string]string{ string(pk): string(pkVal), string(k1): string(k1Val), diff --git a/store/tikv/async_commit_fail_test.go b/store/tikv/async_commit_fail_test.go index 2ded260a50964..837c81789337e 100644 --- a/store/tikv/async_commit_fail_test.go +++ b/store/tikv/async_commit_fail_test.go @@ -26,28 +26,17 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/store/mockstore/cluster" - "github.com/pingcap/tidb/store/mockstore/unistore" ) type testAsyncCommitFailSuite struct { OneByOneSuite testAsyncCommitCommon - cluster cluster.Cluster - store *tikvStore } var _ = SerialSuites(&testAsyncCommitFailSuite{}) func (s *testAsyncCommitFailSuite) SetUpTest(c *C) { - client, pdClient, cluster, err := unistore.New("") - c.Assert(err, IsNil) - unistore.BootstrapWithSingleStore(cluster) - s.cluster = cluster - store, err := NewTestTiKVStore(client, pdClient, nil, nil, 0) - c.Assert(err, IsNil) - - s.store = store.(*tikvStore) + s.testAsyncCommitCommon.setUpTest(c) } // TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when @@ -85,13 +74,47 @@ func (s *testAsyncCommitFailSuite) TestFailAsyncCommitPrewriteRpcErrors(c *C) { c.Assert(bytes.Equal(res, []byte("a1")), IsTrue) } +func (s *testAsyncCommitFailSuite) TestPointGetWithAsyncCommit(c *C) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.Enable = true + }) + + s.putAlphabets(c) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + txn.Set([]byte("a"), []byte("v1")) + txn.Set([]byte("b"), []byte("v2")) + s.mustPointGet(c, []byte("a"), []byte("a")) + s.mustPointGet(c, []byte("b"), []byte("b")) + + // PointGet cannot ignore async commit transactions' locks. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", "return"), IsNil) + ctx := context.WithValue(context.Background(), sessionctx.ConnID, uint64(1)) + err = txn.Commit(ctx) + c.Assert(err, IsNil) + c.Assert(txn.(*tikvTxn).committer.isAsyncCommit(), IsTrue) + s.mustPointGet(c, []byte("a"), []byte("v1")) + s.mustPointGet(c, []byte("b"), []byte("v2")) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing"), IsNil) + + // PointGet will not push the `max_ts` to its ts which is MaxUint64. + txn2, err := s.store.Begin() + c.Assert(err, IsNil) + s.mustGetFromTxn(c, txn2, []byte("a"), []byte("v1")) + s.mustGetFromTxn(c, txn2, []byte("b"), []byte("v2")) + err = txn2.Rollback() + c.Assert(err, IsNil) +} + func (s *testAsyncCommitFailSuite) TestSecondaryListInPrimaryLock(c *C) { defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.Enable = true }) - s.putAlphabets(c, s.store) + s.putAlphabets(c) // Split into several regions. for _, splitKey := range []string{"h", "o", "u"} { diff --git a/store/tikv/async_commit_test.go b/store/tikv/async_commit_test.go index fcd452f9d2dde..b3cfff13c5378 100644 --- a/store/tikv/async_commit_test.go +++ b/store/tikv/async_commit_test.go @@ -24,24 +24,21 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) -type testAsyncCommitCommon struct{} - -type testAsyncCommitSuite struct { - OneByOneSuite - testAsyncCommitCommon +// testAsyncCommitCommon is used to put common parts that will be both used by +// testAsyncCommitSuite and testAsyncCommitFailSuite. +type testAsyncCommitCommon struct { cluster cluster.Cluster store *tikvStore - bo *Backoffer } -var _ = Suite(&testAsyncCommitSuite{}) - -func (s *testAsyncCommitSuite) SetUpTest(c *C) { +func (s *testAsyncCommitCommon) setUpTest(c *C) { client, pdClient, cluster, err := unistore.New("") c.Assert(err, IsNil) unistore.BootstrapWithSingleStore(cluster) @@ -50,17 +47,16 @@ func (s *testAsyncCommitSuite) SetUpTest(c *C) { c.Assert(err, IsNil) s.store = store.(*tikvStore) - s.bo = NewBackofferWithVars(context.Background(), 5000, nil) } -func (s *testAsyncCommitCommon) putAlphabets(c *C, store *tikvStore) { +func (s *testAsyncCommitCommon) putAlphabets(c *C) { for ch := byte('a'); ch <= byte('z'); ch++ { - s.putKV(c, store, []byte{ch}, []byte{ch}) + s.putKV(c, []byte{ch}, []byte{ch}) } } -func (s *testAsyncCommitCommon) putKV(c *C, store *tikvStore, key, value []byte) (uint64, uint64) { - txn, err := store.Begin() +func (s *testAsyncCommitCommon) putKV(c *C, key, value []byte) (uint64, uint64) { + txn, err := s.store.Begin() c.Assert(err, IsNil) err = txn.Set(key, value) c.Assert(err, IsNil) @@ -69,6 +65,53 @@ func (s *testAsyncCommitCommon) putKV(c *C, store *tikvStore, key, value []byte) return txn.StartTS(), txn.(*tikvTxn).commitTS } +func (s *testAsyncCommitCommon) mustGetFromTxn(c *C, txn kv.Transaction, key, expectedValue []byte) { + v, err := txn.Get(context.Background(), key) + c.Assert(err, IsNil) + c.Assert(v, BytesEquals, expectedValue) +} + +func (s *testAsyncCommitCommon) mustGetLock(c *C, key []byte) *Lock { + ver, err := s.store.CurrentVersion() + c.Assert(err, IsNil) + req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{ + Key: key, + Version: ver.Ver, + }) + bo := NewBackofferWithVars(context.Background(), 5000, nil) + loc, err := s.store.regionCache.LocateKey(bo, key) + c.Assert(err, IsNil) + resp, err := s.store.SendReq(bo, req, loc.Region, readTimeoutShort) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + keyErr := resp.Resp.(*kvrpcpb.GetResponse).GetError() + c.Assert(keyErr, NotNil) + lock, err := extractLockFromKeyErr(keyErr) + c.Assert(err, IsNil) + return lock +} + +func (s *testAsyncCommitCommon) mustPointGet(c *C, key, expectedValue []byte) { + snap, err := s.store.GetSnapshot(kv.MaxVersion) + c.Assert(err, IsNil) + value, err := snap.Get(context.Background(), key) + c.Assert(err, IsNil) + c.Assert(value, BytesEquals, expectedValue) +} + +type testAsyncCommitSuite struct { + OneByOneSuite + testAsyncCommitCommon + bo *Backoffer +} + +var _ = Suite(&testAsyncCommitSuite{}) + +func (s *testAsyncCommitSuite) SetUpTest(c *C) { + s.testAsyncCommitCommon.setUpTest(c) + s.bo = NewBackofferWithVars(context.Background(), 5000, nil) +} + func (s *testAsyncCommitSuite) lockKeys(c *C, keys, values [][]byte, primaryKey, primaryValue []byte, commitPrimary bool) (uint64, uint64) { txn, err := newTiKVTxn(s.store) c.Assert(err, IsNil) @@ -103,32 +146,13 @@ func (s *testAsyncCommitSuite) lockKeys(c *C, keys, values [][]byte, primaryKey, return txn.startTS, tpc.commitTS } -func (s *testAsyncCommitSuite) mustGetLock(c *C, key []byte) *Lock { - ver, err := s.store.CurrentVersion() - c.Assert(err, IsNil) - req := tikvrpc.NewRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{ - Key: key, - Version: ver.Ver, - }) - loc, err := s.store.regionCache.LocateKey(s.bo, key) - c.Assert(err, IsNil) - resp, err := s.store.SendReq(s.bo, req, loc.Region, readTimeoutShort) - c.Assert(err, IsNil) - c.Assert(resp.Resp, NotNil) - keyErr := resp.Resp.(*kvrpcpb.GetResponse).GetError() - c.Assert(keyErr, NotNil) - lock, err := extractLockFromKeyErr(keyErr) - c.Assert(err, IsNil) - return lock -} - func (s *testAsyncCommitSuite) TestCheckSecondaries(c *C) { defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.Enable = true }) - s.putAlphabets(c, s.store) + s.putAlphabets(c) loc, err := s.store.GetRegionCache().LocateKey(s.bo, []byte("a")) c.Assert(err, IsNil) @@ -269,6 +293,52 @@ func (s *testAsyncCommitSuite) TestCheckSecondaries(c *C) { c.Assert(gotOther, Equals, int64(0)) } +func (s *testAsyncCommitSuite) TestRepeatableRead(c *C) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.Enable = true + }) + + var connID uint64 = 0 + test := func(isPessimistic bool) { + s.putKV(c, []byte("k1"), []byte("v1")) + + connID++ + ctx := context.WithValue(context.Background(), sessionctx.ConnID, connID) + txn1, err := s.store.Begin() + txn1.SetOption(kv.Pessimistic, isPessimistic) + c.Assert(err, IsNil) + s.mustGetFromTxn(c, txn1, []byte("k1"), []byte("v1")) + txn1.Set([]byte("k1"), []byte("v2")) + + for i := 0; i < 20; i++ { + _, err := s.store.GetOracle().GetTimestamp(ctx) + c.Assert(err, IsNil) + } + + txn2, err := s.store.Begin() + c.Assert(err, IsNil) + s.mustGetFromTxn(c, txn2, []byte("k1"), []byte("v1")) + + err = txn1.Commit(ctx) + c.Assert(err, IsNil) + // Check txn1 is committed in async commit. + c.Assert(txn1.(*tikvTxn).committer.isAsyncCommit(), IsTrue) + s.mustGetFromTxn(c, txn2, []byte("k1"), []byte("v1")) + err = txn2.Rollback() + c.Assert(err, IsNil) + + txn3, err := s.store.Begin() + c.Assert(err, IsNil) + s.mustGetFromTxn(c, txn3, []byte("k1"), []byte("v2")) + err = txn3.Rollback() + c.Assert(err, IsNil) + } + + test(false) + test(true) +} + type mockResolveClient struct { inner Client onResolveLock func(*kvrpcpb.ResolveLockRequest) (*tikvrpc.Response, error) diff --git a/store/tikv/client.go b/store/tikv/client.go index cc69766f18a83..44756f83840b2 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -16,6 +16,7 @@ package tikv import ( "context" + "fmt" "io" "math" "runtime/trace" @@ -42,6 +43,7 @@ import ( "google.golang.org/grpc/backoff" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/credentials" + "google.golang.org/grpc/encoding/gzip" "google.golang.org/grpc/keepalive" ) @@ -134,6 +136,11 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint keepAliveTimeout := cfg.TiKVClient.GrpcKeepAliveTimeout for i := range a.v { ctx, cancel := context.WithTimeout(context.Background(), a.dialTimeout) + var callOptions []grpc.CallOption + callOptions = append(callOptions, grpc.MaxCallRecvMsgSize(MaxRecvMsgSize)) + if cfg.TiKVClient.GrpcCompressionType == gzip.Name { + callOptions = append(callOptions, grpc.UseCompressor(gzip.Name)) + } conn, err := grpc.DialContext( ctx, addr, @@ -142,7 +149,7 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize), grpc.WithUnaryInterceptor(unaryInterceptor), grpc.WithStreamInterceptor(streamInterceptor), - grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(MaxRecvMsgSize)), + grpc.WithDefaultCallOptions(callOptions...), grpc.WithConnectParams(grpc.ConnectParams{ Backoff: backoff.Config{ BaseDelay: 100 * time.Millisecond, // Default was 1s. @@ -319,7 +326,7 @@ func (c *rpcClient) updateTiKVSendReqHistogram(req *tikvrpc.Request, start time. // SendRequest sends a Request to server and receives Response. func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("rpcClient.SendRequest", opentracing.ChildOf(span.Context())) + span1 := span.Tracer().StartSpan(fmt.Sprintf("rpcClient.SendRequest, region ID: %d, type: %s", req.RegionId, req.Type), opentracing.ChildOf(span.Context())) defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 54bb85934f0f2..3fa6fc8c8a3bc 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -807,7 +807,15 @@ func (lr *LockResolver) checkAllSecondaries(bo *Backoffer, l *Lock, status *TxnS return nil, errors.Errorf("async commit recovery (sending CheckSecondaryLocks) finished with errors: %v", errs) } - // TODO(nrc, cfzjywxk) schema lease check + if shared.commitTs > 0 { + schemaVerIsTheSame, err := checkSchemaVersionForAsyncCommit(bo.ctx, l.TxnID, shared.commitTs, lr.store) + if err != nil { + return nil, errors.Trace(err) + } + if !schemaVerIsTheSame { + shared.commitTs = 0 + } + } return &shared, nil } diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 447f6b1fad986..a4f10998b49c5 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -217,7 +217,7 @@ func (s *RegionRequestSender) SendReqCtx( err error, ) { if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan("regionReqauest.SendReqCtx", opentracing.ChildOf(span.Context())) + span1 := span.Tracer().StartSpan("regionRequest.SendReqCtx", opentracing.ChildOf(span.Context())) defer span1.Finish() bo = bo.Clone() bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 401900a74dc60..be33ed4ab341c 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -45,6 +45,7 @@ var ( tikvTxnCmdHistogramWithRollback = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblRollback) tikvTxnCmdHistogramWithBatchGet = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblBatchGet) tikvTxnCmdHistogramWithGet = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblGet) + tikvTxnCmdHistogramWithLockKeys = metrics.TiKVTxnCmdHistogram.WithLabelValues(metrics.LblLockKeys) ) // SchemaAmender is used by pessimistic transactions to amend commit mutations for schema change during 2pc. @@ -368,6 +369,7 @@ func (txn *tikvTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput txn.mu.Lock() defer txn.mu.Unlock() defer func() { + tikvTxnCmdHistogramWithLockKeys.Observe(time.Since(startTime).Seconds()) if err == nil { if lockCtx.PessimisticLockWaited != nil { if atomic.LoadInt32(lockCtx.PessimisticLockWaited) > 0 { diff --git a/table/column.go b/table/column.go index 4fe3f3bf5be0b..783be21341d7e 100644 --- a/table/column.go +++ b/table/column.go @@ -393,13 +393,7 @@ func CheckNotNull(cols []*Column, row []types.Datum) error { // GetColOriginDefaultValue gets default value of the column from original default value. func GetColOriginDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo) (types.Datum, error) { - // If the column type is BIT, both `OriginDefaultValue` and `DefaultValue` of ColumnInfo are corrupted, because - // after JSON marshaling and unmarshaling against the field with type `interface{}`, the content with actual type `[]byte` is changed. - // We need `DefaultValueBit` to restore OriginDefaultValue before reading it. - if col.Tp == mysql.TypeBit && col.DefaultValueBit != nil && col.OriginDefaultValue != nil { - col.OriginDefaultValue = col.DefaultValueBit - } - return getColDefaultValue(ctx, col, col.OriginDefaultValue) + return getColDefaultValue(ctx, col, col.GetOriginDefaultValue()) } // GetColDefaultValue gets default value of the column. diff --git a/table/tables/tables.go b/table/tables/tables.go index c75dedc0f2fab..67eb1e9723fdb 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -442,7 +442,7 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, oldLen := size - 1 colSize[col.ID] = int64(newLen - oldLen) } - sessVars.TxnCtx.UpdateDeltaForTable(t.physicalTableID, 0, 1, colSize) + sessVars.TxnCtx.UpdateDeltaForTable(t.tableID, t.physicalTableID, 0, 1, colSize, sessVars.UseDynamicPartitionPrune()) return nil } @@ -786,7 +786,7 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . } colSize[col.ID] = int64(size) - 1 } - sessVars.TxnCtx.UpdateDeltaForTable(t.physicalTableID, 1, 1, colSize) + sessVars.TxnCtx.UpdateDeltaForTable(t.tableID, t.physicalTableID, 1, 1, colSize, sessVars.UseDynamicPartitionPrune()) return recordID, nil } @@ -1011,7 +1011,7 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type } colSize[col.ID] = -int64(size - 1) } - ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.physicalTableID, -1, 1, colSize) + ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.tableID, t.physicalTableID, -1, 1, colSize, ctx.GetSessionVars().UseDynamicPartitionPrune()) return err } @@ -1253,7 +1253,7 @@ func tryDecodeColumnFromCommonHandle(col *table.Column, handle kv.Handle, pkIds // The defaultVals is used to avoid calculating the default value multiple times. func GetColDefaultValue(ctx sessionctx.Context, col *table.Column, defaultVals []types.Datum) ( colVal types.Datum, err error) { - if col.OriginDefaultValue == nil && mysql.HasNotNullFlag(col.Flag) { + if col.GetOriginDefaultValue() == nil && mysql.HasNotNullFlag(col.Flag) { return colVal, errors.New("Miss column") } if col.State != model.StatePublic { diff --git a/types/convert.go b/types/convert.go index 02d2c0f187cf9..f3f8d291aeedf 100644 --- a/types/convert.go +++ b/types/convert.go @@ -654,6 +654,9 @@ func getValidFloatPrefix(sc *stmtctx.StatementContext, s string, isFuncCast bool break } eIdx = i + } else if c == '\u0000' { + s = s[:validLen] + break } else if c < '0' || c > '9' { break } else { diff --git a/types/convert_test.go b/types/convert_test.go index 7d5147f0f7de0..47d4fa0612e78 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -867,6 +867,7 @@ func (s *testTypeConvertSuite) TestGetValidFloat(c *C) { {"123.e", "123."}, {"0-123", "0"}, {"9-3", "9"}, + {"1001001\\u0000\\u0000\\u0000", "1001001"}, } sc := new(stmtctx.StatementContext) for _, tt := range tests { diff --git a/types/datum.go b/types/datum.go index 03982a0b171e3..464d213a040ec 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1380,9 +1380,14 @@ func (d *Datum) convertToMysqlBit(sc *stmtctx.StatementContext, target *FieldTyp uintDatum, err1 := d.convertToUint(sc, target) uintValue, err = uintDatum.GetUint64(), err1 } - if target.Flen < 64 && uintValue >= 1<<(uint64(target.Flen)) { + // Avoid byte size panic, never goto this branch. + if target.Flen <= 0 || target.Flen >= 128 { return Datum{}, errors.Trace(ErrDataTooLong.GenWithStack("Data Too Long, field len %d", target.Flen)) } + if target.Flen < 64 && uintValue >= 1<<(uint64(target.Flen)) { + uintValue &= (1 << (uint64(target.Flen))) - 1 + err = ErrDataTooLong.GenWithStack("Data Too Long, field len %d", target.Flen) + } byteSize := (target.Flen + 7) >> 3 ret.SetMysqlBit(NewBinaryLiteralFromUint(uintValue, byteSize)) return ret, errors.Trace(err) diff --git a/util/admin/admin.go b/util/admin/admin.go index 902936722d79d..a10045cc13b4d 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -363,7 +363,7 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table for i, val := range vals1 { col := cols[i] if val.IsNull() { - if mysql.HasNotNullFlag(col.Flag) && col.ToInfo().OriginDefaultValue == nil { + if mysql.HasNotNullFlag(col.Flag) && col.ToInfo().GetOriginDefaultValue() == nil { return false, errors.Errorf("Column %v define as not null, but can't find the value where handle is %v", col.Name, h1) } // NULL value is regarded as its default value. diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index ddd01ada18b97..304d9d6c95202 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -128,12 +128,12 @@ func renewEmpty(chk *Chunk) *Chunk { return newChk } -// MemoryUsage returns the total memory usage of a Chunk in B. +// MemoryUsage returns the total memory usage of a Chunk in bytes. // We ignore the size of Column.length and Column.nullCount // since they have little effect of the total memory usage. func (c *Chunk) MemoryUsage() (sum int64) { for _, col := range c.columns { - curColMemUsage := int64(unsafe.Sizeof(*col)) + int64(cap(col.nullBitmap)) + int64(cap(col.offsets)*4) + int64(cap(col.data)) + int64(cap(col.elemBuf)) + curColMemUsage := int64(unsafe.Sizeof(*col)) + int64(cap(col.nullBitmap)) + int64(cap(col.offsets)*8) + int64(cap(col.data)) + int64(cap(col.elemBuf)) sum += curColMemUsage } return @@ -695,3 +695,14 @@ func (c *Chunk) Reconstruct() { c.numVirtualRows = len(c.sel) c.sel = nil } + +// ToString returns all the values in a chunk. +func (c *Chunk) ToString(ft []*types.FieldType) string { + var buf []byte + for rowIdx := 0; rowIdx < c.NumRows(); rowIdx++ { + row := c.GetRow(rowIdx) + buf = append(buf, row.ToString(ft)...) + buf = append(buf, '\n') + } + return string(buf) +} diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 68ef2bad196f9..7bd09ca1e02e7 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -560,11 +560,11 @@ func (s *testChunkSuite) TestChunkMemoryUsage(c *check.C) { initCap := 10 chk := NewChunkWithCapacity(fieldTypes, initCap) - //cap(c.nullBitmap) + cap(c.offsets)*4 + cap(c.data) + cap(c.elemBuf) + //cap(c.nullBitmap) + cap(c.offsets)*8 + cap(c.data) + cap(c.elemBuf) colUsage := make([]int, len(fieldTypes)) colUsage[0] = (initCap+7)>>3 + 0 + initCap*4 + 4 - colUsage[1] = (initCap+7)>>3 + (initCap+1)*4 + initCap*8 + 0 - colUsage[2] = (initCap+7)>>3 + (initCap+1)*4 + initCap*8 + 0 + colUsage[1] = (initCap+7)>>3 + (initCap+1)*8 + initCap*8 + 0 + colUsage[2] = (initCap+7)>>3 + (initCap+1)*8 + initCap*8 + 0 colUsage[3] = (initCap+7)>>3 + 0 + initCap*sizeTime + sizeTime colUsage[4] = (initCap+7)>>3 + 0 + initCap*8 + 8 @@ -596,7 +596,7 @@ func (s *testChunkSuite) TestChunkMemoryUsage(c *check.C) { chk.AppendDuration(4, durationObj) memUsage = chk.MemoryUsage() - colUsage[1] = (initCap+7)>>3 + (initCap+1)*4 + cap(chk.columns[1].data) + 0 + colUsage[1] = (initCap+7)>>3 + (initCap+1)*8 + cap(chk.columns[1].data) + 0 expectedUsage = 0 for i := range colUsage { expectedUsage += colUsage[i] + int(unsafe.Sizeof(*chk.columns[i])) @@ -786,6 +786,30 @@ func (s *testChunkSuite) TestMakeRefTo(c *check.C) { c.Assert(chk2.columns[1] == chk1.columns[0], check.IsTrue) } +func (s *testChunkSuite) TestToString(c *check.C) { + fieldTypes := make([]*types.FieldType, 0, 4) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeFloat}) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeDouble}) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeString}) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeDate}) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeLonglong}) + + chk := NewChunkWithCapacity(fieldTypes, 2) + chk.AppendFloat32(0, float32(1)) + chk.AppendFloat64(1, 1.0) + chk.AppendString(2, "1") + chk.AppendTime(3, types.ZeroDate) + chk.AppendInt64(4, 1) + + chk.AppendFloat32(0, float32(2)) + chk.AppendFloat64(1, 2.0) + chk.AppendString(2, "2") + chk.AppendTime(3, types.ZeroDatetime) + chk.AppendInt64(4, 2) + + c.Assert(chk.ToString(fieldTypes), check.Equals, "1, 1, 1, 0000-00-00, 1\n2, 2, 2, 0000-00-00 00:00:00, 2\n") +} + func BenchmarkAppendInt(b *testing.B) { b.ReportAllocs() chk := newChunk(8) diff --git a/util/chunk/row.go b/util/chunk/row.go index 5b9bf41684f54..993ec9b58b9d1 100644 --- a/util/chunk/row.go +++ b/util/chunk/row.go @@ -14,6 +14,8 @@ package chunk import ( + "strconv" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" @@ -213,3 +215,46 @@ func (r Row) CopyConstruct() Row { newChk.AppendRow(r) return newChk.GetRow(0) } + +// ToString returns all the values in a row. +func (r Row) ToString(ft []*types.FieldType) string { + var buf []byte + for colIdx := 0; colIdx < r.Chunk().NumCols(); colIdx++ { + if r.IsNull(colIdx) { + buf = append(buf, "nil, "...) + continue + } + switch ft[colIdx].EvalType() { + case types.ETInt: + buf = strconv.AppendInt(buf, r.GetInt64(colIdx), 10) + case types.ETString: + switch ft[colIdx].Tp { + case mysql.TypeEnum: + buf = append(buf, r.GetEnum(colIdx).String()...) + case mysql.TypeSet: + buf = append(buf, r.GetSet(colIdx).String()...) + default: + buf = append(buf, r.GetString(colIdx)...) + } + case types.ETDatetime, types.ETTimestamp: + buf = append(buf, r.GetTime(colIdx).String()...) + case types.ETDecimal: + buf = append(buf, r.GetMyDecimal(colIdx).ToString()...) + case types.ETDuration: + buf = append(buf, r.GetDuration(colIdx, ft[colIdx].Decimal).String()...) + case types.ETJson: + buf = append(buf, r.GetJSON(colIdx).String()...) + case types.ETReal: + switch ft[colIdx].Tp { + case mysql.TypeFloat: + buf = strconv.AppendFloat(buf, float64(r.GetFloat32(colIdx)), 'f', -1, 32) + case mysql.TypeDouble: + buf = strconv.AppendFloat(buf, r.GetFloat64(colIdx), 'f', -1, 64) + } + } + if colIdx != r.Chunk().NumCols()-1 { + buf = append(buf, ", "...) + } + } + return string(buf) +} diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 5b7c28e2d8909..7ebe67f3890ea 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -432,6 +432,8 @@ const ( TpSelectResultRuntimeStats // TpInsertRuntimeStat is the tp for InsertRuntimeStat TpInsertRuntimeStat + // TpSlowQueryRuntimeStat is the tp for TpSlowQueryRuntimeStat + TpSlowQueryRuntimeStat ) // RuntimeStats is used to express the executor runtime information. diff --git a/util/memory/tracker.go b/util/memory/tracker.go index a0edbdf912c80..d5ad74031ae5a 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -302,6 +302,11 @@ func (t *Tracker) toString(indent string, buffer *bytes.Buffer) { // BytesToString converts the memory consumption to a readable string. func (t *Tracker) BytesToString(numBytes int64) string { + return BytesToString(numBytes) +} + +// BytesToString converts the memory consumption to a readable string. +func BytesToString(numBytes int64) string { GB := float64(numBytes) / float64(1<<30) if GB > 1 { return fmt.Sprintf("%v GB", GB)