diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 509ec590b20b2..c8436799feff0 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -681,11 +681,11 @@ func TestRuntimeHintsInEvolveTasks(t *testing.T) { tk.MustExec("create table t(a int, b int, c int, index idx_a(a), index idx_b(b), index idx_c(c))") tk.MustExec("create global binding for select * from t where a >= 1 and b >= 1 and c = 0 using select * from t use index(idx_a) where a >= 1 and b >= 1 and c = 0") - tk.MustQuery("select /*+ MAX_EXECUTION_TIME(5000) */ * from t where a >= 4 and b >= 1 and c = 0") + tk.MustQuery("select /*+ MAX_EXECUTION_TIME(5000), TIDB_KV_READ_TIMEOUT(20) */ * from t where a >= 4 and b >= 1 and c = 0") tk.MustExec("admin flush bindings") rows := tk.MustQuery("show global bindings").Rows() require.Len(t, rows, 2) - require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idx_c`), max_execution_time(5000)*/ * FROM `test`.`t` WHERE `a` >= 4 AND `b` >= 1 AND `c` = 0", rows[0][1]) + require.Equal(t, "SELECT /*+ use_index(@`sel_1` `test`.`t` `idx_c`), no_order_index(@`sel_1` `test`.`t` `idx_c`), max_execution_time(5000), tidb_kv_read_timeout(20)*/ * FROM `test`.`t` WHERE `a` >= 4 AND `b` >= 1 AND `c` = 0", rows[0][1]) } func TestDefaultSessionVars(t *testing.T) { @@ -745,13 +745,15 @@ func TestStmtHints(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, index idx(a))") - tk.MustExec("create global binding for select * from t using select /*+ MAX_EXECUTION_TIME(100), MEMORY_QUOTA(2 GB) */ * from t use index(idx)") + tk.MustExec("create global binding for select * from t using select /*+ MAX_EXECUTION_TIME(100), TIDB_KV_READ_TIMEOUT(20), MEMORY_QUOTA(2 GB) */ * from t use index(idx)") tk.MustQuery("select * from t") require.Equal(t, int64(2147483648), tk.Session().GetSessionVars().MemTracker.GetBytesLimit()) require.Equal(t, uint64(100), tk.Session().GetSessionVars().StmtCtx.MaxExecutionTime) + require.Equal(t, uint64(20), tk.Session().GetSessionVars().StmtCtx.TidbKvReadTimeout) tk.MustQuery("select a, b from t") require.Equal(t, int64(1073741824), tk.Session().GetSessionVars().MemTracker.GetBytesLimit()) require.Equal(t, uint64(0), tk.Session().GetSessionVars().StmtCtx.MaxExecutionTime) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().StmtCtx.TidbKvReadTimeout) } func TestPrivileges(t *testing.T) { @@ -1284,6 +1286,7 @@ func TestBindSQLDigest(t *testing.T) { // runtime hints {"select * from t", "select /*+ memory_quota(1024 MB) */ * from t"}, {"select * from t", "select /*+ max_execution_time(1000) */ * from t"}, + {"select * from t", "select /*+ tidb_kv_read_timeout(1000) */ * from t"}, // storage hints {"select * from t", "select /*+ read_from_storage(tikv[t]) */ * from t"}, // others @@ -1345,6 +1348,7 @@ func TestDropBindBySQLDigest(t *testing.T) { // runtime hints {"select * from t", "select /*+ memory_quota(1024 MB) */ * from t"}, {"select * from t", "select /*+ max_execution_time(1000) */ * from t"}, + {"select * from t", "select /*+ tidb_kv_read_timeout(1000) */ * from t"}, // storage hints {"select * from t", "select /*+ read_from_storage(tikv[t]) */ * from t"}, // others diff --git a/bindinfo/capture_test.go b/bindinfo/capture_test.go index d1f375a6b63d7..c92bdf380cd38 100644 --- a/bindinfo/capture_test.go +++ b/bindinfo/capture_test.go @@ -999,6 +999,7 @@ func TestCaptureHints(t *testing.T) { // runtime hints {"select /*+ memory_quota(1024 MB) */ * from t", "memory_quota(1024 mb)"}, {"select /*+ max_execution_time(1000) */ * from t", "max_execution_time(1000)"}, + {"select /*+ tidb_kv_read_timeout(1000) */ * from t", "tidb_kv_read_timeout(1000)"}, // storage hints {"select /*+ read_from_storage(tikv[t]) */ * from t", "read_from_storage(tikv[`t`])"}, // others diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 700366636cc79..7b7ad7a249d7d 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -301,6 +301,7 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req builder.RequestSource.RequestSourceInternal = sv.InRestrictedSQL builder.RequestSource.RequestSourceType = sv.RequestSourceType builder.StoreBatchSize = sv.StoreBatchSize + builder.Request.TidbKvReadTimeout = sv.GetTidbKvReadTimeout() return builder } diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 8e35bb85d5f57..bd4299e237ca3 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -598,6 +598,33 @@ func TestRequestBuilder8(t *testing.T) { require.Equal(t, expect, actual) } +func TestRequestBuilderTidbKvReadTimeout(t *testing.T) { + sv := variable.NewSessionVars(nil) + sv.TidbKvReadTimeout = 100 + actual, err := (&RequestBuilder{}). + SetFromSessionVars(sv). + Build() + require.NoError(t, err) + expect := &kv.Request{ + Tp: 0, + StartTs: 0x0, + Data: []uint8(nil), + KeyRanges: kv.NewNonParitionedKeyRanges(nil), + Concurrency: variable.DefDistSQLScanConcurrency, + IsolationLevel: 0, + Priority: 0, + MemTracker: (*memory.Tracker)(nil), + SchemaVar: 0, + ReadReplicaScope: kv.GlobalReplicaScope, + TidbKvReadTimeout: 100, + ResourceGroupName: resourcegroup.DefaultResourceGroupName, + } + expect.Paging.MinPagingSize = paging.MinPagingSize + expect.Paging.MaxPagingSize = paging.MaxPagingSize + actual.ResourceGroupTagger = nil + require.Equal(t, expect, actual) +} + func TestTableRangesToKVRangesWithFbs(t *testing.T) { ranges := []*ranger.Range{ { diff --git a/executor/adapter.go b/executor/adapter.go index 3360f9b8898d1..59d7164df443c 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1729,7 +1729,8 @@ func getEncodedPlan(stmtCtx *stmtctx.StatementContext, genHint bool) (encodedPla // some hints like 'memory_quota' cannot be extracted from the PhysicalPlan directly, // so we have to iterate all hints from the customer and keep some other necessary hints. switch tableHint.HintName.L { - case "memory_quota", "use_toja", "no_index_merge", "max_execution_time", + case plannercore.HintMemoryQuota, plannercore.HintUseToja, plannercore.HintNoIndexMerge, + plannercore.HintMaxExecutionTime, plannercore.HintTidbKvReadTimeout, plannercore.HintAggToCop, plannercore.HintIgnoreIndex, plannercore.HintReadFromStorage, plannercore.HintLimitToCop: hints = append(hints, tableHint) diff --git a/executor/builder.go b/executor/builder.go index 582a0fd0c75e3..b8b76326c5bac 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1771,6 +1771,7 @@ func (b *executorBuilder) getSnapshot() (kv.Snapshot, error) { replicaReadType := sessVars.GetReplicaRead() snapshot.SetOption(kv.ReadReplicaScope, b.readReplicaScope) snapshot.SetOption(kv.TaskID, sessVars.StmtCtx.TaskID) + snapshot.SetOption(kv.TidbKvReadTimeout, sessVars.GetTidbKvReadTimeout()) if replicaReadType.IsClosestRead() && b.readReplicaScope != kv.GlobalTxnScope { snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{ diff --git a/executor/executor_failpoint_test.go b/executor/executor_failpoint_test.go index 0e72dccd1e15c..9311ad4ff4531 100644 --- a/executor/executor_failpoint_test.go +++ b/executor/executor_failpoint_test.go @@ -555,3 +555,70 @@ func TestDeadlocksTable(t *testing.T) { id2+"/2022-06-11 02:03:04.987654/1/203/////201", )) } + +func TestTidbKvReadTimeout(t *testing.T) { + if *testkit.WithTiKV != "" { + t.Skip("skip test since it's only work for unistore") + } + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int primary key, b int)") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCDeadlineExceeded", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCDeadlineExceeded")) + }() + // Test for point_get request + rows := tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t where a = 1").Rows() + require.Len(t, rows, 1) + explain := fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*Point_Get.* Get:{num_rpc:2, total_time:.*", explain) + + // Test for batch_point_get request + rows = tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t where a in (1,2)").Rows() + require.Len(t, rows, 1) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*Batch_Point_Get.* BatchGet:{num_rpc:2, total_time:.*", explain) + + // Test for cop request + rows = tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain) + + // Test for stale read. + tk.MustExec("set @a=now(6);") + tk.MustExec("set @@tidb_replica_read='closest-replicas';") + rows = tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t as of timestamp(@a) where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain) + + // Test for tidb_kv_read_timeout session variable. + tk.MustExec("set @@tidb_kv_read_timeout=1;") + // Test for point_get request + rows = tk.MustQuery("explain analyze select * from t where a = 1").Rows() + require.Len(t, rows, 1) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*Point_Get.* Get:{num_rpc:2, total_time:.*", explain) + + // Test for batch_point_get request + rows = tk.MustQuery("explain analyze select * from t where a in (1,2)").Rows() + require.Len(t, rows, 1) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*Batch_Point_Get.* BatchGet:{num_rpc:2, total_time:.*", explain) + + // Test for cop request + rows = tk.MustQuery("explain analyze select * from t where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain) + + // Test for stale read. + tk.MustExec("set @a=now(6);") + tk.MustExec("set @@tidb_replica_read='closest-replicas';") + rows = tk.MustQuery("explain analyze select * from t as of timestamp(@a) where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain) +} diff --git a/go.mod b/go.mod index 016ad40f8f632..384001a7d7efd 100644 --- a/go.mod +++ b/go.mod @@ -270,3 +270,5 @@ replace ( github.com/pingcap/tidb/parser => ./parser go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) + +replace github.com/tikv/client-go/v2 => ../client-go diff --git a/kv/kv.go b/kv/kv.go index d84c34d1b89c4..e37cfafa3fac1 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -545,6 +545,8 @@ type Request struct { StoreBatchSize int // LimitSize indicates whether the request is scan and limit LimitSize uint64 + // TidbKvReadTimeout is the timeout of kv read request + TidbKvReadTimeout uint64 } // CoprRequestAdjuster is used to check and adjust a copr request according to specific rules. diff --git a/kv/option.go b/kv/option.go index 2ac479eedaf49..57e67071fd44c 100644 --- a/kv/option.go +++ b/kv/option.go @@ -100,6 +100,8 @@ const ( // and the next 8 bits are reserved for Lossy DDL reorg Backfill job. // The remaining 48 bits are reserved for extendability. TxnSource + // TidbKvReadTimeout sets the timeout value for readonly kv request in milliseconds + TidbKvReadTimeout ) // ReplicaReadType is the type of replica to read data from diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 32f8c7b27a7ed..1ff8812a0534f 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -3721,6 +3721,8 @@ func (n *TableOptimizerHint) Restore(ctx *format.RestoreCtx) error { switch n.HintName.L { case "max_execution_time": ctx.WritePlainf("%d", n.HintData.(uint64)) + case "tidb_kv_read_timeout": + ctx.WritePlainf("%d", n.HintData.(uint64)) case "nth_plan": ctx.WritePlainf("%d", n.HintData.(int64)) case "tidb_hj", "tidb_smj", "tidb_inlj", "hash_join", "hash_join_build", "hash_join_probe", "merge_join", "inl_join", "broadcast_join", "shuffle_join", "inl_hash_join", "inl_merge_join", "leading": diff --git a/parser/ast/misc_test.go b/parser/ast/misc_test.go index 7379279d51567..3d44069a95683 100644 --- a/parser/ast/misc_test.go +++ b/parser/ast/misc_test.go @@ -270,6 +270,8 @@ func TestTableOptimizerHintRestore(t *testing.T) { {"LEADING(t1@sel1, c1, t2)", "LEADING(`t1`@`sel1`, `c1`, `t2`)"}, {"MAX_EXECUTION_TIME(3000)", "MAX_EXECUTION_TIME(3000)"}, {"MAX_EXECUTION_TIME(@sel1 3000)", "MAX_EXECUTION_TIME(@`sel1` 3000)"}, + {"TIDB_KV_READ_TIMEOUT(3000)", "TIDB_KV_READ_TIMEOUT(3000)"}, + {"TIDB_KV_READ_TIMEOUT(@sel1 3000)", "TIDB_KV_READ_TIMEOUT(@`sel1` 3000)"}, {"USE_INDEX_MERGE(t1 c1)", "USE_INDEX_MERGE(`t1` `c1`)"}, {"USE_INDEX_MERGE(@sel1 t1 c1)", "USE_INDEX_MERGE(@`sel1` `t1` `c1`)"}, {"USE_INDEX_MERGE(t1@sel1 c1)", "USE_INDEX_MERGE(`t1`@`sel1` `c1`)"}, diff --git a/parser/hintparser.go b/parser/hintparser.go index ec266f90556d9..7bdf10fa20076 100644 --- a/parser/hintparser.go +++ b/parser/hintparser.go @@ -41,235 +41,237 @@ type yyhintXError struct { } const ( - yyhintDefault = 57433 + yyhintDefault = 57434 yyhintEOFCode = 57344 yyhintErrCode = 57345 - hintAggToCop = 57379 - hintBCJoin = 57401 + hintAggToCop = 57380 + hintBCJoin = 57402 hintBKA = 57355 hintBNL = 57357 - hintDupsWeedOut = 57429 - hintFalse = 57425 - hintFirstMatch = 57430 - hintForceIndex = 57415 - hintGB = 57428 - hintHashAgg = 57381 + hintDupsWeedOut = 57430 + hintFalse = 57426 + hintFirstMatch = 57431 + hintForceIndex = 57416 + hintGB = 57429 + hintHashAgg = 57382 hintHashJoin = 57359 hintHashJoinBuild = 57360 hintHashJoinProbe = 57361 hintIdentifier = 57347 - hintIgnoreIndex = 57384 - hintIgnorePlanCache = 57380 - hintIndexHashJoin = 57388 - hintIndexJoin = 57385 + hintIgnoreIndex = 57385 + hintIgnorePlanCache = 57381 + hintIndexHashJoin = 57389 + hintIndexJoin = 57386 hintIndexMerge = 57365 - hintIndexMergeJoin = 57392 - hintInlHashJoin = 57387 - hintInlJoin = 57390 - hintInlMergeJoin = 57391 + hintIndexMergeJoin = 57393 + hintInlHashJoin = 57388 + hintInlJoin = 57391 + hintInlMergeJoin = 57392 hintIntLit = 57346 hintInvalid = 57348 hintJoinFixedOrder = 57351 hintJoinOrder = 57352 hintJoinPrefix = 57353 hintJoinSuffix = 57354 - hintLeading = 57417 - hintLimitToCop = 57414 - hintLooseScan = 57431 - hintMB = 57427 + hintLeading = 57418 + hintLimitToCop = 57415 + hintLooseScan = 57432 + hintMB = 57428 hintMRR = 57367 - hintMaterialization = 57432 + hintMaterialization = 57433 hintMaxExecutionTime = 57375 - hintMemoryQuota = 57394 + hintMemoryQuota = 57395 hintMerge = 57363 - hintMpp1PhaseAgg = 57382 - hintMpp2PhaseAgg = 57383 + hintMpp1PhaseAgg = 57383 + hintMpp2PhaseAgg = 57384 hintNoBKA = 57356 hintNoBNL = 57358 - hintNoDecorrelate = 57419 + hintNoDecorrelate = 57420 hintNoHashJoin = 57362 hintNoICP = 57369 - hintNoIndexHashJoin = 57389 - hintNoIndexJoin = 57386 + hintNoIndexHashJoin = 57390 + hintNoIndexJoin = 57387 hintNoIndexMerge = 57366 - hintNoIndexMergeJoin = 57393 + hintNoIndexMergeJoin = 57394 hintNoMRR = 57368 hintNoMerge = 57364 - hintNoOrderIndex = 57408 + hintNoOrderIndex = 57409 hintNoRangeOptimization = 57370 - hintNoSMJoin = 57400 + hintNoSMJoin = 57401 hintNoSemijoin = 57374 hintNoSkipScan = 57372 - hintNoSwapJoinInputs = 57395 - hintNthPlan = 57413 - hintOLAP = 57420 - hintOLTP = 57421 - hintOrderIndex = 57407 - hintPartition = 57422 - hintQBName = 57378 - hintQueryType = 57396 - hintReadConsistentReplica = 57397 - hintReadFromStorage = 57398 - hintResourceGroup = 57377 - hintSMJoin = 57399 - hintSemiJoinRewrite = 57418 + hintNoSwapJoinInputs = 57396 + hintNthPlan = 57414 + hintOLAP = 57421 + hintOLTP = 57422 + hintOrderIndex = 57408 + hintPartition = 57423 + hintQBName = 57379 + hintQueryType = 57397 + hintReadConsistentReplica = 57398 + hintReadFromStorage = 57399 + hintResourceGroup = 57378 + hintSMJoin = 57400 + hintSemiJoinRewrite = 57419 hintSemijoin = 57373 - hintSetVar = 57376 - hintShuffleJoin = 57402 + hintSetVar = 57377 + hintShuffleJoin = 57403 hintSingleAtIdentifier = 57349 hintSkipScan = 57371 - hintStraightJoin = 57416 - hintStreamAgg = 57403 + hintStraightJoin = 57417 + hintStreamAgg = 57404 hintStringLit = 57350 - hintSwapJoinInputs = 57404 - hintTiFlash = 57424 - hintTiKV = 57423 - hintTimeRange = 57411 - hintTrue = 57426 - hintUseCascades = 57412 - hintUseIndex = 57406 - hintUseIndexMerge = 57405 - hintUsePlanCache = 57409 - hintUseToja = 57410 + hintSwapJoinInputs = 57405 + hintTiFlash = 57425 + hintTiKV = 57424 + hintTidbKvReadTimeout = 57376 + hintTimeRange = 57412 + hintTrue = 57427 + hintUseCascades = 57413 + hintUseIndex = 57407 + hintUseIndexMerge = 57406 + hintUsePlanCache = 57410 + hintUseToja = 57411 yyhintMaxDepth = 200 - yyhintTabOfs = -211 + yyhintTabOfs = -213 ) var ( yyhintXLAT = map[int]int{ - 41: 0, // ')' (155x) - 57379: 1, // hintAggToCop (142x) - 57401: 2, // hintBCJoin (142x) - 57355: 3, // hintBKA (142x) - 57357: 4, // hintBNL (142x) - 57415: 5, // hintForceIndex (142x) - 57381: 6, // hintHashAgg (142x) - 57359: 7, // hintHashJoin (142x) - 57360: 8, // hintHashJoinBuild (142x) - 57361: 9, // hintHashJoinProbe (142x) - 57384: 10, // hintIgnoreIndex (142x) - 57380: 11, // hintIgnorePlanCache (142x) - 57388: 12, // hintIndexHashJoin (142x) - 57385: 13, // hintIndexJoin (142x) - 57365: 14, // hintIndexMerge (142x) - 57392: 15, // hintIndexMergeJoin (142x) - 57387: 16, // hintInlHashJoin (142x) - 57390: 17, // hintInlJoin (142x) - 57391: 18, // hintInlMergeJoin (142x) - 57351: 19, // hintJoinFixedOrder (142x) - 57352: 20, // hintJoinOrder (142x) - 57353: 21, // hintJoinPrefix (142x) - 57354: 22, // hintJoinSuffix (142x) - 57417: 23, // hintLeading (142x) - 57414: 24, // hintLimitToCop (142x) - 57375: 25, // hintMaxExecutionTime (142x) - 57394: 26, // hintMemoryQuota (142x) - 57363: 27, // hintMerge (142x) - 57382: 28, // hintMpp1PhaseAgg (142x) - 57383: 29, // hintMpp2PhaseAgg (142x) - 57367: 30, // hintMRR (142x) - 57356: 31, // hintNoBKA (142x) - 57358: 32, // hintNoBNL (142x) - 57419: 33, // hintNoDecorrelate (142x) - 57362: 34, // hintNoHashJoin (142x) - 57369: 35, // hintNoICP (142x) - 57389: 36, // hintNoIndexHashJoin (142x) - 57386: 37, // hintNoIndexJoin (142x) - 57366: 38, // hintNoIndexMerge (142x) - 57393: 39, // hintNoIndexMergeJoin (142x) - 57364: 40, // hintNoMerge (142x) - 57368: 41, // hintNoMRR (142x) - 57408: 42, // hintNoOrderIndex (142x) - 57370: 43, // hintNoRangeOptimization (142x) - 57374: 44, // hintNoSemijoin (142x) - 57372: 45, // hintNoSkipScan (142x) - 57400: 46, // hintNoSMJoin (142x) - 57395: 47, // hintNoSwapJoinInputs (142x) - 57413: 48, // hintNthPlan (142x) - 57407: 49, // hintOrderIndex (142x) - 57378: 50, // hintQBName (142x) - 57396: 51, // hintQueryType (142x) - 57397: 52, // hintReadConsistentReplica (142x) - 57398: 53, // hintReadFromStorage (142x) - 57377: 54, // hintResourceGroup (142x) - 57373: 55, // hintSemijoin (142x) - 57418: 56, // hintSemiJoinRewrite (142x) - 57376: 57, // hintSetVar (142x) - 57402: 58, // hintShuffleJoin (142x) - 57371: 59, // hintSkipScan (142x) - 57399: 60, // hintSMJoin (142x) - 57416: 61, // hintStraightJoin (142x) - 57403: 62, // hintStreamAgg (142x) - 57404: 63, // hintSwapJoinInputs (142x) - 57411: 64, // hintTimeRange (142x) - 57412: 65, // hintUseCascades (142x) - 57406: 66, // hintUseIndex (142x) - 57405: 67, // hintUseIndexMerge (142x) - 57409: 68, // hintUsePlanCache (142x) - 57410: 69, // hintUseToja (142x) - 44: 70, // ',' (139x) - 57429: 71, // hintDupsWeedOut (119x) - 57430: 72, // hintFirstMatch (119x) - 57431: 73, // hintLooseScan (119x) - 57432: 74, // hintMaterialization (119x) - 57424: 75, // hintTiFlash (119x) - 57423: 76, // hintTiKV (119x) - 57425: 77, // hintFalse (118x) - 57420: 78, // hintOLAP (118x) - 57421: 79, // hintOLTP (118x) - 57426: 80, // hintTrue (118x) - 57428: 81, // hintGB (117x) - 57427: 82, // hintMB (117x) - 57347: 83, // hintIdentifier (116x) - 57349: 84, // hintSingleAtIdentifier (102x) - 93: 85, // ']' (93x) - 46: 86, // '.' (92x) - 57422: 87, // hintPartition (87x) - 61: 88, // '=' (83x) - 40: 89, // '(' (78x) - 57344: 90, // $end (25x) - 57453: 91, // QueryBlockOpt (20x) - 57445: 92, // Identifier (15x) - 57346: 93, // hintIntLit (8x) - 57350: 94, // hintStringLit (5x) - 57435: 95, // CommaOpt (4x) - 57441: 96, // HintTable (4x) - 57442: 97, // HintTableList (4x) - 91: 98, // '[' (3x) - 57434: 99, // BooleanHintName (2x) - 57436: 100, // HintIndexList (2x) - 57438: 101, // HintStorageType (2x) - 57439: 102, // HintStorageTypeAndTable (2x) - 57443: 103, // HintTableListOpt (2x) - 57448: 104, // JoinOrderOptimizerHintName (2x) - 57449: 105, // NullaryHintName (2x) - 57452: 106, // PartitionListOpt (2x) - 57455: 107, // StorageOptimizerHintOpt (2x) - 57456: 108, // SubqueryOptimizerHintName (2x) - 57459: 109, // SubqueryStrategy (2x) - 57460: 110, // SupportedIndexLevelOptimizerHintName (2x) - 57461: 111, // SupportedTableLevelOptimizerHintName (2x) - 57462: 112, // TableOptimizerHintOpt (2x) - 57464: 113, // UnsupportedIndexLevelOptimizerHintName (2x) - 57465: 114, // UnsupportedTableLevelOptimizerHintName (2x) - 57467: 115, // ViewName (2x) - 57437: 116, // HintQueryType (1x) - 57440: 117, // HintStorageTypeAndTableList (1x) - 57444: 118, // HintTrueOrFalse (1x) - 57446: 119, // IndexNameList (1x) - 57447: 120, // IndexNameListOpt (1x) - 57450: 121, // OptimizerHintList (1x) - 57451: 122, // PartitionList (1x) - 57454: 123, // Start (1x) - 57457: 124, // SubqueryStrategies (1x) - 57458: 125, // SubqueryStrategiesOpt (1x) - 57463: 126, // UnitOfBytes (1x) - 57466: 127, // Value (1x) - 57468: 128, // ViewNameList (1x) - 57433: 129, // $default (0x) - 57345: 130, // error (0x) - 57348: 131, // hintInvalid (0x) + 41: 0, // ')' (157x) + 57380: 1, // hintAggToCop (144x) + 57402: 2, // hintBCJoin (144x) + 57355: 3, // hintBKA (144x) + 57357: 4, // hintBNL (144x) + 57416: 5, // hintForceIndex (144x) + 57382: 6, // hintHashAgg (144x) + 57359: 7, // hintHashJoin (144x) + 57360: 8, // hintHashJoinBuild (144x) + 57361: 9, // hintHashJoinProbe (144x) + 57385: 10, // hintIgnoreIndex (144x) + 57381: 11, // hintIgnorePlanCache (144x) + 57389: 12, // hintIndexHashJoin (144x) + 57386: 13, // hintIndexJoin (144x) + 57365: 14, // hintIndexMerge (144x) + 57393: 15, // hintIndexMergeJoin (144x) + 57388: 16, // hintInlHashJoin (144x) + 57391: 17, // hintInlJoin (144x) + 57392: 18, // hintInlMergeJoin (144x) + 57351: 19, // hintJoinFixedOrder (144x) + 57352: 20, // hintJoinOrder (144x) + 57353: 21, // hintJoinPrefix (144x) + 57354: 22, // hintJoinSuffix (144x) + 57418: 23, // hintLeading (144x) + 57415: 24, // hintLimitToCop (144x) + 57375: 25, // hintMaxExecutionTime (144x) + 57395: 26, // hintMemoryQuota (144x) + 57363: 27, // hintMerge (144x) + 57383: 28, // hintMpp1PhaseAgg (144x) + 57384: 29, // hintMpp2PhaseAgg (144x) + 57367: 30, // hintMRR (144x) + 57356: 31, // hintNoBKA (144x) + 57358: 32, // hintNoBNL (144x) + 57420: 33, // hintNoDecorrelate (144x) + 57362: 34, // hintNoHashJoin (144x) + 57369: 35, // hintNoICP (144x) + 57390: 36, // hintNoIndexHashJoin (144x) + 57387: 37, // hintNoIndexJoin (144x) + 57366: 38, // hintNoIndexMerge (144x) + 57394: 39, // hintNoIndexMergeJoin (144x) + 57364: 40, // hintNoMerge (144x) + 57368: 41, // hintNoMRR (144x) + 57409: 42, // hintNoOrderIndex (144x) + 57370: 43, // hintNoRangeOptimization (144x) + 57374: 44, // hintNoSemijoin (144x) + 57372: 45, // hintNoSkipScan (144x) + 57401: 46, // hintNoSMJoin (144x) + 57396: 47, // hintNoSwapJoinInputs (144x) + 57414: 48, // hintNthPlan (144x) + 57408: 49, // hintOrderIndex (144x) + 57379: 50, // hintQBName (144x) + 57397: 51, // hintQueryType (144x) + 57398: 52, // hintReadConsistentReplica (144x) + 57399: 53, // hintReadFromStorage (144x) + 57378: 54, // hintResourceGroup (144x) + 57373: 55, // hintSemijoin (144x) + 57419: 56, // hintSemiJoinRewrite (144x) + 57377: 57, // hintSetVar (144x) + 57403: 58, // hintShuffleJoin (144x) + 57371: 59, // hintSkipScan (144x) + 57400: 60, // hintSMJoin (144x) + 57417: 61, // hintStraightJoin (144x) + 57404: 62, // hintStreamAgg (144x) + 57405: 63, // hintSwapJoinInputs (144x) + 57376: 64, // hintTidbKvReadTimeout (144x) + 57412: 65, // hintTimeRange (144x) + 57413: 66, // hintUseCascades (144x) + 57407: 67, // hintUseIndex (144x) + 57406: 68, // hintUseIndexMerge (144x) + 57410: 69, // hintUsePlanCache (144x) + 57411: 70, // hintUseToja (144x) + 44: 71, // ',' (141x) + 57430: 72, // hintDupsWeedOut (120x) + 57431: 73, // hintFirstMatch (120x) + 57432: 74, // hintLooseScan (120x) + 57433: 75, // hintMaterialization (120x) + 57425: 76, // hintTiFlash (120x) + 57424: 77, // hintTiKV (120x) + 57426: 78, // hintFalse (119x) + 57421: 79, // hintOLAP (119x) + 57422: 80, // hintOLTP (119x) + 57427: 81, // hintTrue (119x) + 57429: 82, // hintGB (118x) + 57428: 83, // hintMB (118x) + 57347: 84, // hintIdentifier (117x) + 57349: 85, // hintSingleAtIdentifier (104x) + 93: 86, // ']' (94x) + 46: 87, // '.' (93x) + 57423: 88, // hintPartition (88x) + 61: 89, // '=' (84x) + 40: 90, // '(' (79x) + 57344: 91, // $end (26x) + 57454: 92, // QueryBlockOpt (21x) + 57446: 93, // Identifier (15x) + 57346: 94, // hintIntLit (10x) + 57350: 95, // hintStringLit (5x) + 57436: 96, // CommaOpt (4x) + 57442: 97, // HintTable (4x) + 57443: 98, // HintTableList (4x) + 91: 99, // '[' (3x) + 57435: 100, // BooleanHintName (2x) + 57437: 101, // HintIndexList (2x) + 57439: 102, // HintStorageType (2x) + 57440: 103, // HintStorageTypeAndTable (2x) + 57444: 104, // HintTableListOpt (2x) + 57449: 105, // JoinOrderOptimizerHintName (2x) + 57450: 106, // NullaryHintName (2x) + 57453: 107, // PartitionListOpt (2x) + 57456: 108, // StorageOptimizerHintOpt (2x) + 57457: 109, // SubqueryOptimizerHintName (2x) + 57460: 110, // SubqueryStrategy (2x) + 57461: 111, // SupportedIndexLevelOptimizerHintName (2x) + 57462: 112, // SupportedTableLevelOptimizerHintName (2x) + 57463: 113, // TableOptimizerHintOpt (2x) + 57465: 114, // UnsupportedIndexLevelOptimizerHintName (2x) + 57466: 115, // UnsupportedTableLevelOptimizerHintName (2x) + 57468: 116, // ViewName (2x) + 57438: 117, // HintQueryType (1x) + 57441: 118, // HintStorageTypeAndTableList (1x) + 57445: 119, // HintTrueOrFalse (1x) + 57447: 120, // IndexNameList (1x) + 57448: 121, // IndexNameListOpt (1x) + 57451: 122, // OptimizerHintList (1x) + 57452: 123, // PartitionList (1x) + 57455: 124, // Start (1x) + 57458: 125, // SubqueryStrategies (1x) + 57459: 126, // SubqueryStrategiesOpt (1x) + 57464: 127, // UnitOfBytes (1x) + 57467: 128, // Value (1x) + 57469: 129, // ViewNameList (1x) + 57434: 130, // $default (0x) + 57345: 131, // error (0x) + 57348: 132, // hintInvalid (0x) } yyhintSymNames = []string{ @@ -337,6 +339,7 @@ var ( "hintStraightJoin", "hintStreamAgg", "hintSwapJoinInputs", + "hintTidbKvReadTimeout", "hintTimeRange", "hintUseCascades", "hintUseIndex", @@ -409,70 +412,99 @@ var ( yyhintReductions = []struct{ xsym, components int }{ {0, 1}, - {123, 1}, - {121, 1}, - {121, 3}, - {121, 1}, - {121, 3}, - {112, 4}, - {112, 4}, - {112, 4}, - {112, 4}, - {112, 4}, - {112, 4}, - {112, 5}, - {112, 5}, - {112, 5}, - {112, 6}, - {112, 4}, - {112, 4}, - {112, 6}, - {112, 6}, - {112, 6}, - {112, 5}, - {112, 4}, - {112, 5}, - {107, 5}, - {117, 1}, - {117, 3}, - {102, 4}, - {91, 0}, - {91, 1}, - {95, 0}, - {95, 1}, - {106, 0}, - {106, 4}, + {124, 1}, + {122, 1}, + {122, 3}, {122, 1}, {122, 3}, - {103, 1}, - {103, 1}, - {97, 2}, + {113, 4}, + {113, 4}, + {113, 4}, + {113, 4}, + {113, 4}, + {113, 4}, + {113, 5}, + {113, 5}, + {113, 5}, + {113, 5}, + {113, 6}, + {113, 4}, + {113, 4}, + {113, 6}, + {113, 6}, + {113, 6}, + {113, 5}, + {113, 4}, + {113, 5}, + {108, 5}, + {118, 1}, + {118, 3}, + {103, 4}, + {92, 0}, + {92, 1}, + {96, 0}, + {96, 1}, + {107, 0}, + {107, 4}, + {123, 1}, + {123, 3}, + {104, 1}, + {104, 1}, + {98, 2}, + {98, 3}, {97, 3}, - {96, 3}, - {96, 5}, - {128, 3}, - {128, 1}, - {115, 2}, - {115, 1}, - {100, 4}, - {120, 0}, + {97, 5}, + {129, 3}, + {129, 1}, + {116, 2}, + {116, 1}, + {101, 4}, + {121, 0}, + {121, 1}, {120, 1}, - {119, 1}, - {119, 3}, - {125, 0}, + {120, 3}, + {126, 0}, + {126, 1}, {125, 1}, - {124, 1}, - {124, 3}, - {127, 1}, + {125, 3}, + {128, 1}, + {128, 1}, + {128, 1}, {127, 1}, {127, 1}, - {126, 1}, - {126, 1}, - {118, 1}, - {118, 1}, - {104, 1}, - {104, 1}, - {104, 1}, + {119, 1}, + {119, 1}, + {105, 1}, + {105, 1}, + {105, 1}, + {115, 1}, + {115, 1}, + {115, 1}, + {115, 1}, + {115, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {112, 1}, + {114, 1}, + {114, 1}, {114, 1}, {114, 1}, {114, 1}, @@ -484,504 +516,484 @@ var ( {111, 1}, {111, 1}, {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {111, 1}, - {113, 1}, - {113, 1}, - {113, 1}, - {113, 1}, - {113, 1}, - {113, 1}, - {113, 1}, - {110, 1}, - {110, 1}, + {109, 1}, + {109, 1}, {110, 1}, {110, 1}, {110, 1}, {110, 1}, - {108, 1}, - {108, 1}, - {109, 1}, - {109, 1}, - {109, 1}, - {109, 1}, - {99, 1}, - {99, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {116, 1}, - {116, 1}, - {101, 1}, - {101, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, - {92, 1}, + {100, 1}, + {100, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {117, 1}, + {117, 1}, + {102, 1}, + {102, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, + {93, 1}, } yyhintXErrors = map[yyhintXError]string{} - yyhintParseTab = [298][]uint16{ + yyhintParseTab = [304][]uint16{ // 0 - {1: 285, 244, 237, 239, 273, 281, 258, 260, 261, 271, 289, 251, 247, 263, 256, 250, 246, 255, 216, 234, 235, 236, 262, 286, 223, 228, 249, 282, 283, 264, 238, 240, 292, 259, 266, 252, 248, 287, 257, 241, 265, 275, 267, 277, 269, 243, 254, 224, 274, 227, 232, 288, 233, 226, 276, 291, 225, 245, 268, 242, 290, 284, 253, 229, 279, 270, 272, 280, 278, 99: 230, 104: 217, 231, 107: 215, 222, 110: 221, 219, 214, 220, 218, 121: 213, 123: 212}, - {90: 211}, - {1: 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 397, 90: 210, 95: 506}, - {1: 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 90: 209}, - {1: 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 90: 207}, + {1: 288, 247, 240, 242, 276, 284, 261, 263, 264, 274, 292, 254, 250, 266, 259, 253, 249, 258, 218, 237, 238, 239, 265, 289, 225, 231, 252, 285, 286, 267, 241, 243, 295, 262, 269, 255, 251, 290, 260, 244, 268, 278, 270, 280, 272, 246, 257, 227, 277, 230, 235, 291, 236, 229, 279, 294, 228, 248, 271, 245, 293, 287, 256, 226, 232, 282, 273, 275, 283, 281, 100: 233, 105: 219, 234, 108: 217, 224, 111: 223, 221, 216, 222, 220, 122: 215, 124: 214}, + {91: 213}, + {1: 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 401, 91: 212, 96: 514}, + {1: 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 211, 91: 211}, + {1: 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 209, 91: 209}, // 5 - {89: 503}, - {89: 500}, - {89: 497}, - {89: 492}, - {89: 489}, + {90: 511}, + {90: 508}, + {90: 505}, + {90: 500}, + {90: 497}, // 10 - {89: 478}, - {89: 466}, - {89: 462}, - {89: 458}, - {89: 450}, + {90: 486}, + {90: 474}, + {90: 470}, + {90: 466}, + {90: 462}, // 15 - {89: 447}, - {89: 435}, - {89: 428}, - {89: 423}, - {89: 417}, + {90: 454}, + {90: 451}, + {90: 439}, + {90: 432}, + {90: 427}, // 20 - {89: 414}, - {89: 408}, - {89: 293}, - {89: 149}, - {89: 148}, + {90: 421}, + {90: 418}, + {90: 412}, + {90: 296}, + {90: 150}, // 25 - {89: 147}, - {89: 146}, - {89: 145}, - {89: 144}, - {89: 143}, + {90: 149}, + {90: 148}, + {90: 147}, + {90: 146}, + {90: 145}, // 30 - {89: 142}, - {89: 141}, - {89: 140}, - {89: 139}, - {89: 138}, + {90: 144}, + {90: 143}, + {90: 142}, + {90: 141}, + {90: 140}, // 35 - {89: 137}, - {89: 136}, - {89: 135}, - {89: 134}, - {89: 133}, + {90: 139}, + {90: 138}, + {90: 137}, + {90: 136}, + {90: 135}, // 40 - {89: 132}, - {89: 131}, - {89: 130}, - {89: 129}, - {89: 128}, + {90: 134}, + {90: 133}, + {90: 132}, + {90: 131}, + {90: 130}, // 45 - {89: 127}, - {89: 126}, - {89: 125}, - {89: 124}, - {89: 123}, + {90: 129}, + {90: 128}, + {90: 127}, + {90: 126}, + {90: 125}, // 50 - {89: 122}, - {89: 121}, - {89: 120}, - {89: 119}, - {89: 118}, + {90: 124}, + {90: 123}, + {90: 122}, + {90: 121}, + {90: 120}, // 55 - {89: 117}, - {89: 116}, - {89: 115}, - {89: 114}, - {89: 113}, + {90: 119}, + {90: 118}, + {90: 117}, + {90: 116}, + {90: 115}, // 60 - {89: 112}, - {89: 111}, - {89: 110}, - {89: 109}, - {89: 108}, + {90: 114}, + {90: 113}, + {90: 112}, + {90: 111}, + {90: 110}, // 65 - {89: 107}, - {89: 106}, - {89: 101}, - {89: 100}, - {89: 99}, + {90: 109}, + {90: 108}, + {90: 107}, + {90: 102}, + {90: 101}, // 70 - {89: 98}, - {89: 97}, - {89: 96}, - {89: 95}, - {89: 94}, + {90: 100}, + {90: 99}, + {90: 98}, + {90: 97}, + {90: 96}, // 75 - {89: 93}, - {89: 92}, - {89: 91}, - {89: 90}, - {89: 89}, + {90: 95}, + {90: 94}, + {90: 93}, + {90: 92}, + {90: 91}, // 80 - {89: 88}, - {89: 87}, - {75: 183, 183, 84: 295, 91: 294}, - {75: 300, 299, 101: 298, 297, 117: 296}, - {182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 85: 182, 182, 182, 93: 182}, + {90: 90}, + {90: 89}, + {90: 88}, + {76: 184, 184, 85: 298, 92: 297}, + {76: 303, 302, 102: 301, 300, 118: 299}, // 85 - {405, 70: 406}, - {186, 70: 186}, - {98: 301}, - {98: 84}, - {98: 83}, + {183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 86: 183, 183, 183, 94: 183}, + {409, 71: 410}, + {187, 71: 187}, + {99: 304}, + {99: 85}, // 90 - {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 71: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 91: 303, 97: 302}, - {70: 403, 85: 402}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 305, 96: 304}, - {173, 70: 173, 85: 173}, - {183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 183, 389, 183, 91: 388}, + {99: 84}, + {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 72: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 92: 306, 98: 305}, + {71: 407, 86: 406}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 308, 97: 307}, + {174, 71: 174, 86: 174}, // 95 - {82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82}, - {81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81}, - {80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80}, - {79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79}, - {78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78}, + {184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 184, 393, 184, 92: 392}, + {83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83, 83}, + {82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82, 82}, + {81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81, 81}, + {80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80}, // 100 - {77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77}, - {76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76}, - {75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75}, - {74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74}, - {73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73}, + {79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79, 79}, + {78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78, 78}, + {77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77, 77}, + {76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76}, + {75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75}, // 105 - {72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72}, - {71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71}, - {70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70}, - {69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69}, - {68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68}, + {74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74}, + {73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73}, + {72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72}, + {71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71}, + {70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70}, // 110 - {67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, - {66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66}, - {65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65}, - {64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64}, - {63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63}, + {69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69}, + {68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68}, + {67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, + {66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66}, + {65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65}, // 115 - {62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62}, - {61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61}, - {60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60}, - {59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59}, - {58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58}, + {64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64}, + {63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63}, + {62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62}, + {61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61}, + {60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60}, // 120 - {57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57}, - {56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56}, - {55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55}, - {54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54}, - {53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53}, + {59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59}, + {58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58}, + {57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57}, + {56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56}, + {55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55}, // 125 - {52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52}, - {51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51}, - {50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50}, - {49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49}, - {48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48}, + {54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54}, + {53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53}, + {52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52}, + {51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51}, + {50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50}, // 130 - {47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47}, - {46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46}, - {45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45}, - {44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44}, - {43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43}, + {49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49}, + {48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48}, + {47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47}, + {46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46}, + {45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45}, // 135 - {42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42}, - {41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41}, - {40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40}, - {39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39}, - {38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38}, + {44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44}, + {43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43}, + {42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42}, + {41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41}, + {40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40}, // 140 - {37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37}, - {36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36}, - {35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35}, - {34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34}, - {33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33}, + {39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39}, + {38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38}, + {37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37}, + {36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36}, + {35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35}, // 145 - {32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32}, - {31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31}, - {30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30}, - {29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29}, - {28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28}, + {34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34}, + {33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33}, + {32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32}, + {31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31}, + {30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30}, // 150 - {27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27}, - {26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26}, - {25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25}, - {24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24}, - {23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23}, + {29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29}, + {28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28}, + {27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27}, + {26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26}, + {25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25}, // 155 - {22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22}, - {21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21}, - {20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20}, - {19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19}, - {18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18}, + {24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24}, + {23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23}, + {22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22}, + {21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21}, + {20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20}, // 160 - {17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17}, - {16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16}, - {15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15}, - {14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14}, - {13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13}, + {19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19}, + {18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18}, + {17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17}, + {16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16}, + {15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15}, // 165 - {12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12}, - {11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11}, - {10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10}, - {9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9}, - {8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8}, + {14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14}, + {13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13}, + {12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12}, + {11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11}, + {10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10}, // 170 - {7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7}, - {6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6}, - {5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, - {4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, - {3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3}, + {9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9}, + {8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8}, + {7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7}, + {6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6}, + {5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, // 175 - {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2}, - {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}, - {179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 85: 179, 87: 392, 106: 401}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 390}, - {183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 183, 87: 183, 91: 391}, + {4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, + {3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3}, + {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2}, + {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}, + {180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 86: 180, 88: 396, 107: 405}, // 180 - {179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 85: 179, 87: 392, 106: 393}, - {89: 394}, - {170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 85: 170}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 396, 122: 395}, - {398, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 397, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 95: 399}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 394}, + {184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 184, 88: 184, 92: 395}, + {180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 86: 180, 88: 396, 107: 397}, + {90: 398}, + {171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 86: 171}, // 185 - {177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177}, - {180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 71: 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 94: 180}, - {178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 85: 178}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 400}, - {176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 400, 123: 399}, + {402, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 401, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 96: 403}, + {178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178}, + {181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 72: 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 95: 181}, + {179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 86: 179}, // 190 - {171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 85: 171}, - {184, 70: 184}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 305, 96: 404}, - {172, 70: 172, 85: 172}, - {1: 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 90: 187}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 404}, + {177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177}, + {172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 86: 172}, + {185, 71: 185}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 308, 97: 408}, // 195 - {75: 300, 299, 101: 298, 407}, - {185, 70: 185}, - {78: 183, 183, 84: 295, 91: 409}, - {78: 411, 412, 116: 410}, - {413}, + {173, 71: 173, 86: 173}, + {1: 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 91: 188}, + {76: 303, 302, 102: 301, 411}, + {186, 71: 186}, + {79: 184, 184, 85: 298, 92: 413}, // 200 + {79: 415, 416, 117: 414}, + {417}, + {87}, {86}, - {85}, - {1: 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 90: 188}, - {183, 84: 295, 91: 415}, - {416}, + {1: 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 91: 189}, // 205 - {1: 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 90: 189}, - {77: 183, 80: 183, 84: 295, 91: 418}, - {77: 421, 80: 420, 118: 419}, - {422}, - {151}, + {184, 85: 298, 92: 419}, + {420}, + {1: 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 91: 190}, + {78: 184, 81: 184, 85: 298, 92: 422}, + {78: 425, 81: 424, 119: 423}, // 210 - {150}, - {1: 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 90: 190}, - {94: 424}, - {70: 397, 94: 181, 425}, - {94: 426}, + {426}, + {152}, + {151}, + {1: 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 91: 191}, + {95: 428}, // 215 - {427}, - {1: 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 90: 191}, - {84: 295, 91: 429, 93: 183}, - {93: 430}, - {81: 433, 432, 126: 431}, + {71: 401, 95: 182, 429}, + {95: 430}, + {431}, + {1: 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 91: 192}, + {85: 298, 92: 433, 94: 184}, // 220 - {434}, + {94: 434}, + {82: 437, 436, 127: 435}, + {438}, + {154}, {153}, - {152}, - {1: 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 90: 192}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 436}, // 225 - {437, 70: 438}, - {1: 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 90: 194}, - {183, 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 295, 86: 183, 91: 442, 441, 115: 440, 128: 439}, - {444, 86: 445}, - {168, 86: 168}, + {1: 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 91: 193}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 440}, + {441, 71: 442}, + {1: 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 91: 195}, + {184, 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 298, 87: 184, 92: 446, 445, 116: 444, 129: 443}, // 230 - {183, 84: 295, 86: 183, 91: 443}, - {166, 86: 166}, - {167, 86: 167}, - {1: 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 90: 193}, - {183, 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 295, 86: 183, 91: 442, 441, 115: 446}, + {448, 87: 449}, + {169, 87: 169}, + {184, 85: 298, 87: 184, 92: 447}, + {167, 87: 167}, + {168, 87: 168}, // 235 - {169, 86: 169}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 448}, - {449}, - {1: 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 90: 195}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 451}, + {1: 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 91: 194}, + {184, 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 298, 87: 184, 92: 446, 445, 116: 450}, + {170, 87: 170}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 452}, + {453}, // 240 - {88: 452}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 455, 456, 454, 127: 453}, - {457}, + {1: 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 91: 196}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 455}, + {89: 456}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 459, 460, 458, 128: 457}, + {461}, + // 245 + {157}, {156}, {155}, - // 245 - {154}, - {1: 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 196, 90: 196}, - {84: 295, 91: 459, 93: 183}, - {93: 460}, - {461}, + {1: 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 91: 197}, + {85: 298, 92: 463, 94: 184}, // 250 - {1: 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 197, 90: 197}, - {84: 295, 91: 463, 93: 183}, - {93: 464}, + {94: 464}, {465}, - {1: 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 90: 198}, + {1: 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 198, 91: 198}, + {85: 298, 92: 467, 94: 184}, + {94: 468}, // 255 - {183, 71: 183, 183, 183, 183, 84: 295, 91: 467}, - {160, 71: 471, 472, 473, 474, 109: 470, 124: 469, 468}, - {477}, - {159, 70: 475}, - {158, 70: 158}, + {469}, + {1: 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 91: 199}, + {85: 298, 92: 471, 94: 184}, + {94: 472}, + {473}, // 260 - {105, 70: 105}, - {104, 70: 104}, - {103, 70: 103}, - {102, 70: 102}, - {71: 471, 472, 473, 474, 109: 476}, + {1: 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 91: 200}, + {184, 72: 184, 184, 184, 184, 85: 298, 92: 475}, + {161, 72: 479, 480, 481, 482, 110: 478, 125: 477, 476}, + {485}, + {160, 71: 483}, // 265 - {157, 70: 157}, - {1: 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 199, 90: 199}, - {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 71: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 91: 480, 100: 479}, - {488}, - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 305, 96: 481}, + {159, 71: 159}, + {106, 71: 106}, + {105, 71: 105}, + {104, 71: 104}, + {103, 71: 103}, // 270 - {181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 397, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 95: 482}, - {164, 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 485, 119: 484, 483}, - {165}, - {163, 70: 486}, - {162, 70: 162}, + {72: 479, 480, 481, 482, 110: 484}, + {158, 71: 158}, + {1: 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 91: 201}, + {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 72: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 92: 488, 101: 487}, + {496}, // 275 - {1: 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 487}, - {161, 70: 161}, - {1: 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 200, 90: 200}, - {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 71: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 91: 480, 100: 490}, - {491}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 308, 97: 489}, + {182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 401, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 96: 490}, + {165, 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 493, 120: 492, 491}, + {166}, + {164, 71: 494}, // 280 - {1: 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 201, 90: 201}, - {183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 71: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 91: 495, 97: 494, 103: 493}, - {496}, - {175, 70: 403}, - {174, 335, 358, 311, 313, 371, 338, 315, 316, 317, 341, 337, 343, 346, 321, 349, 342, 345, 348, 307, 308, 309, 310, 373, 336, 331, 351, 319, 339, 340, 323, 312, 314, 375, 318, 325, 344, 347, 322, 350, 320, 324, 365, 326, 330, 328, 357, 352, 370, 364, 334, 353, 354, 355, 333, 329, 374, 332, 359, 327, 356, 372, 360, 361, 368, 369, 363, 362, 366, 367, 71: 384, 385, 386, 387, 379, 378, 380, 376, 377, 381, 383, 382, 306, 92: 305, 96: 304}, + {163, 71: 163}, + {1: 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 495}, + {162, 71: 162}, + {1: 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 91: 202}, + {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 72: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 92: 488, 101: 498}, // 285 - {1: 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 202, 90: 202}, - {183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 71: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 91: 495, 97: 494, 103: 498}, {499}, - {1: 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 90: 203}, - {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 71: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 295, 91: 303, 97: 501}, + {1: 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, 91: 203}, + {184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 72: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 92: 503, 98: 502, 104: 501}, + {504}, + {176, 71: 407}, // 290 - {502, 70: 403}, - {1: 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 90: 204}, - {183, 84: 295, 91: 504}, - {505}, - {1: 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 90: 205}, + {175, 338, 361, 314, 316, 374, 341, 318, 319, 320, 344, 340, 346, 349, 324, 352, 345, 348, 351, 310, 311, 312, 313, 376, 339, 334, 354, 322, 342, 343, 326, 315, 317, 378, 321, 328, 347, 350, 325, 353, 323, 327, 368, 329, 333, 331, 360, 355, 373, 367, 337, 356, 357, 358, 336, 332, 377, 335, 362, 330, 359, 375, 363, 364, 379, 371, 372, 366, 365, 369, 370, 72: 388, 389, 390, 391, 383, 382, 384, 380, 381, 385, 387, 386, 309, 93: 308, 97: 307}, + {1: 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, 91: 204}, + {184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 72: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 92: 503, 98: 502, 104: 506}, + {507}, + {1: 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 205, 91: 205}, // 295 - {1: 285, 244, 237, 239, 273, 281, 258, 260, 261, 271, 289, 251, 247, 263, 256, 250, 246, 255, 216, 234, 235, 236, 262, 286, 223, 228, 249, 282, 283, 264, 238, 240, 292, 259, 266, 252, 248, 287, 257, 241, 265, 275, 267, 277, 269, 243, 254, 224, 274, 227, 232, 288, 233, 226, 276, 291, 225, 245, 268, 242, 290, 284, 253, 229, 279, 270, 272, 280, 278, 99: 230, 104: 217, 231, 107: 508, 222, 110: 221, 219, 507, 220, 218}, - {1: 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 90: 208}, - {1: 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 90: 206}, + {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 72: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 298, 92: 306, 98: 509}, + {510, 71: 407}, + {1: 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 206, 91: 206}, + {184, 85: 298, 92: 512}, + {513}, + // 300 + {1: 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 207, 91: 207}, + {1: 288, 247, 240, 242, 276, 284, 261, 263, 264, 274, 292, 254, 250, 266, 259, 253, 249, 258, 218, 237, 238, 239, 265, 289, 225, 231, 252, 285, 286, 267, 241, 243, 295, 262, 269, 255, 251, 290, 260, 244, 268, 278, 270, 280, 272, 246, 257, 227, 277, 230, 235, 291, 236, 229, 279, 294, 228, 248, 271, 245, 293, 287, 256, 226, 232, 282, 273, 275, 283, 281, 100: 233, 105: 219, 234, 108: 516, 224, 111: 223, 221, 515, 222, 220}, + {1: 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 210, 91: 210}, + {1: 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 208, 91: 208}, } ) @@ -1021,7 +1033,7 @@ func yyhintlex1(yylex yyhintLexer, lval *yyhintSymType) (n int) { } func yyhintParse(yylex yyhintLexer, parser *hintParser) int { - const yyError = 130 + const yyError = 131 yyEx, _ := yylex.(yyhintLexerEx) var yyn int @@ -1272,10 +1284,18 @@ yynewstate: parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-4].ident), QBName: model.NewCIStr(yyS[yypt-2].ident), - HintData: int64(yyS[yypt-1].number), + HintData: yyS[yypt-1].number, } } case 15: + { + parser.yyVAL.hint = &ast.TableOptimizerHint{ + HintName: model.NewCIStr(yyS[yypt-4].ident), + QBName: model.NewCIStr(yyS[yypt-2].ident), + HintData: int64(yyS[yypt-1].number), + } + } + case 16: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-5].ident), @@ -1285,19 +1305,19 @@ yynewstate: }, } } - case 16: + case 17: { parser.warnUnsupportedHint(yyS[yypt-3].ident) parser.yyVAL.hint = nil } - case 17: + case 18: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-3].ident), QBName: model.NewCIStr(yyS[yypt-1].ident), } } - case 18: + case 19: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-5].ident), @@ -1305,7 +1325,7 @@ yynewstate: Tables: yyS[yypt-1].hint.Tables, } } - case 19: + case 20: { maxValue := uint64(math.MaxInt64) / yyS[yypt-1].number if yyS[yypt-2].number <= maxValue { @@ -1320,7 +1340,7 @@ yynewstate: parser.yyVAL.hint = nil } } - case 20: + case 21: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-5].ident), @@ -1330,21 +1350,21 @@ yynewstate: }, } } - case 21: + case 22: { h := yyS[yypt-1].hint h.HintName = model.NewCIStr(yyS[yypt-4].ident) h.QBName = model.NewCIStr(yyS[yypt-2].ident) parser.yyVAL.hint = h } - case 22: + case 23: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-3].ident), QBName: model.NewCIStr(yyS[yypt-1].ident), } } - case 23: + case 24: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-4].ident), @@ -1352,7 +1372,7 @@ yynewstate: HintData: model.NewCIStr(yyS[yypt-1].ident), } } - case 24: + case 25: { hs := yyS[yypt-1].hints name := model.NewCIStr(yyS[yypt-4].ident) @@ -1363,60 +1383,60 @@ yynewstate: } parser.yyVAL.hints = hs } - case 25: + case 26: { parser.yyVAL.hints = []*ast.TableOptimizerHint{yyS[yypt-0].hint} } - case 26: + case 27: { parser.yyVAL.hints = append(yyS[yypt-2].hints, yyS[yypt-0].hint) } - case 27: + case 28: { h := yyS[yypt-1].hint h.HintData = model.NewCIStr(yyS[yypt-3].ident) parser.yyVAL.hint = h } - case 28: + case 29: { parser.yyVAL.ident = "" } - case 32: + case 33: { parser.yyVAL.modelIdents = nil } - case 33: + case 34: { parser.yyVAL.modelIdents = yyS[yypt-1].modelIdents } - case 34: + case 35: { parser.yyVAL.modelIdents = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} } - case 35: + case 36: { parser.yyVAL.modelIdents = append(yyS[yypt-2].modelIdents, model.NewCIStr(yyS[yypt-0].ident)) } - case 37: + case 38: { parser.yyVAL.hint = &ast.TableOptimizerHint{ QBName: model.NewCIStr(yyS[yypt-0].ident), } } - case 38: + case 39: { parser.yyVAL.hint = &ast.TableOptimizerHint{ Tables: []ast.HintTable{yyS[yypt-0].table}, QBName: model.NewCIStr(yyS[yypt-1].ident), } } - case 39: + case 40: { h := yyS[yypt-2].hint h.Tables = append(h.Tables, yyS[yypt-0].table) parser.yyVAL.hint = h } - case 40: + case 41: { parser.yyVAL.table = ast.HintTable{ TableName: model.NewCIStr(yyS[yypt-2].ident), @@ -1424,7 +1444,7 @@ yynewstate: PartitionList: yyS[yypt-0].modelIdents, } } - case 41: + case 42: { parser.yyVAL.table = ast.HintTable{ DBName: model.NewCIStr(yyS[yypt-4].ident), @@ -1433,71 +1453,71 @@ yynewstate: PartitionList: yyS[yypt-0].modelIdents, } } - case 42: + case 43: { h := yyS[yypt-2].hint h.Tables = append(h.Tables, yyS[yypt-0].table) parser.yyVAL.hint = h } - case 43: + case 44: { parser.yyVAL.hint = &ast.TableOptimizerHint{ Tables: []ast.HintTable{yyS[yypt-0].table}, } } - case 44: + case 45: { parser.yyVAL.table = ast.HintTable{ TableName: model.NewCIStr(yyS[yypt-1].ident), QBName: model.NewCIStr(yyS[yypt-0].ident), } } - case 45: + case 46: { parser.yyVAL.table = ast.HintTable{ QBName: model.NewCIStr(yyS[yypt-0].ident), } } - case 46: + case 47: { h := yyS[yypt-0].hint h.Tables = []ast.HintTable{yyS[yypt-2].table} h.QBName = model.NewCIStr(yyS[yypt-3].ident) parser.yyVAL.hint = h } - case 47: + case 48: { parser.yyVAL.hint = &ast.TableOptimizerHint{} } - case 49: + case 50: { parser.yyVAL.hint = &ast.TableOptimizerHint{ Indexes: []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)}, } } - case 50: + case 51: { h := yyS[yypt-2].hint h.Indexes = append(h.Indexes, model.NewCIStr(yyS[yypt-0].ident)) parser.yyVAL.hint = h } - case 57: + case 58: { parser.yyVAL.ident = strconv.FormatUint(yyS[yypt-0].number, 10) } - case 58: + case 59: { parser.yyVAL.number = 1024 * 1024 } - case 59: + case 60: { parser.yyVAL.number = 1024 * 1024 * 1024 } - case 60: + case 61: { parser.yyVAL.hint = &ast.TableOptimizerHint{HintData: true} } - case 61: + case 62: { parser.yyVAL.hint = &ast.TableOptimizerHint{HintData: false} } diff --git a/parser/hintparser.y b/parser/hintparser.y index dd21c899017de..3a93cd68f2ded 100644 --- a/parser/hintparser.y +++ b/parser/hintparser.y @@ -77,6 +77,7 @@ import ( hintSemijoin "SEMIJOIN" hintNoSemijoin "NO_SEMIJOIN" hintMaxExecutionTime "MAX_EXECUTION_TIME" + hintTidbKvReadTimeout "TIDB_KV_READ_TIMEOUT" hintSetVar "SET_VAR" hintResourceGroup "RESOURCE_GROUP" hintQBName "QB_NAME" @@ -266,6 +267,14 @@ TableOptimizerHintOpt: HintData: $4, } } +| "TIDB_KV_READ_TIMEOUT" '(' QueryBlockOpt hintIntLit ')' + { + $$ = &ast.TableOptimizerHint{ + HintName: model.NewCIStr($1), + QBName: model.NewCIStr($3), + HintData: $4, + } + } | "NTH_PLAN" '(' QueryBlockOpt hintIntLit ')' { $$ = &ast.TableOptimizerHint{ @@ -736,6 +745,7 @@ Identifier: | "LEADING" | "SEMI_JOIN_REWRITE" | "NO_DECORRELATE" +| "TIDB_KV_READ_TIMEOUT" /* other keywords */ | "OLAP" | "OLTP" diff --git a/parser/misc.go b/parser/misc.go index 024779048a1e6..6740fdde7563f 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -969,6 +969,7 @@ var hintTokenMap = map[string]int{ "LEADING": hintLeading, "SEMI_JOIN_REWRITE": hintSemiJoinRewrite, "NO_DECORRELATE": hintNoDecorrelate, + "TIDB_KV_READ_TIMEOUT": hintTidbKvReadTimeout, // TiDB hint aliases "TIDB_HJ": hintHashJoin, diff --git a/parser/parser_test.go b/parser/parser_test.go index 2b052ca422884..6267da14f5524 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -3698,6 +3698,10 @@ func TestHintError(t *testing.T) { _, warns, err = p.Parse("create global binding for select /*+ max_execution_time(1) */ 1 using select /*+ max_execution_time(1) */ 1;\n", "", "") require.NoError(t, err) require.Len(t, warns, 0) + + _, warns, err = p.Parse("create global binding for select /*+ tidb_kv_read_timeout(1) */ 1 using select /*+ tidb_kv_read_timeout(1) */ 1;\n", "", "") + require.NoError(t, err) + require.Len(t, warns, 0) } func TestErrorMsg(t *testing.T) { @@ -4106,6 +4110,23 @@ func TestOptimizerHints(t *testing.T) { require.Equal(t, uint64(1000), hints[0].HintData.(uint64)) } + // Test TIDB_KV_READ_TIMEOUT + queries = []string{ + "SELECT /*+ TIDB_KV_READ_TIMEOUT(200) */ * FROM t1 INNER JOIN t2 where t1.c1 = t2.c1", + "SELECT /*+ TIDB_KV_READ_TIMEOUT(200) */ 1", + "SELECT /*+ TIDB_KV_READ_TIMEOUT(200) */ SLEEP(20)", + "SELECT /*+ TIDB_KV_READ_TIMEOUT(200) */ 1 FROM DUAL", + } + for i, query := range queries { + stmt, _, err = p.Parse(query, "", "") + require.NoError(t, err) + selectStmt = stmt[0].(*ast.SelectStmt) + hints = selectStmt.TableHints + require.Len(t, hints, 1) + require.Equal(t, "tidb_kv_read_timeout", hints[0].HintName.L, "case", i) + require.Equal(t, uint64(200), hints[0].HintData.(uint64)) + } + // Test NTH_PLAN queries = []string{ "SELECT /*+ NTH_PLAN(10) */ * FROM t1 INNER JOIN t2 where t1.c1 = t2.c1", @@ -6727,6 +6748,7 @@ func TestCTE(t *testing.T) { {"with cte(a) as (select 1) delete t from t, cte where t.a=cte.a;", true, "WITH `cte` (`a`) AS (SELECT 1) DELETE `t` FROM (`t`) JOIN `cte` WHERE `t`.`a`=`cte`.`a`"}, {"WITH cte1 AS (SELECT 1) SELECT * FROM (WITH cte2 AS (SELECT 2) SELECT * FROM cte2 JOIN cte1) AS dt;", true, "WITH `cte1` AS (SELECT 1) SELECT * FROM (WITH `cte2` AS (SELECT 2) SELECT * FROM `cte2` JOIN `cte1`) AS `dt`"}, {"WITH cte AS (SELECT 1) SELECT /*+ MAX_EXECUTION_TIME(1000) */ * FROM cte;", true, "WITH `cte` AS (SELECT 1) SELECT /*+ MAX_EXECUTION_TIME(1000)*/ * FROM `cte`"}, + {"WITH cte AS (SELECT 1) SELECT /*+ TIDB_KV_READ_TIMEOUT(1000) */ * FROM cte;", true, "WITH `cte` AS (SELECT 1) SELECT /*+ TIDB_KV_READ_TIMEOUT(1000)*/ * FROM `cte`"}, {"with cte as (table t) table cte;", true, "WITH `cte` AS (TABLE `t`) TABLE `cte`"}, {"with cte as (select 1) select 1 union with cte as (select 1) select * from cte;", false, ""}, {"with cte as (select 1) (select 1);", true, "WITH `cte` AS (SELECT 1) (SELECT 1)"}, @@ -6751,6 +6773,7 @@ func TestCTEMerge(t *testing.T) { {"with cte(a) as (select 1) delete t from t, cte where t.a=cte.a;", true, "WITH `cte` (`a`) AS (SELECT 1) DELETE `t` FROM (`t`) JOIN `cte` WHERE `t`.`a`=`cte`.`a`"}, {"WITH cte1 AS (SELECT 1) SELECT * FROM (WITH cte2 AS (SELECT 2) SELECT * FROM cte2 JOIN cte1) AS dt;", true, "WITH `cte1` AS (SELECT 1) SELECT * FROM (WITH `cte2` AS (SELECT 2) SELECT * FROM `cte2` JOIN `cte1`) AS `dt`"}, {"WITH cte AS (SELECT 1) SELECT /*+ MAX_EXECUTION_TIME(1000) */ * FROM cte;", true, "WITH `cte` AS (SELECT 1) SELECT /*+ MAX_EXECUTION_TIME(1000)*/ * FROM `cte`"}, + {"WITH cte AS (SELECT 1) SELECT /*+ TIDB_KV_READ_TIMEOUT(1000) */ * FROM cte;", true, "WITH `cte` AS (SELECT 1) SELECT /*+ TIDB_KV_READ_TIMEOUT(1000)*/ * FROM `cte`"}, {"with cte as (table t) table cte;", true, "WITH `cte` AS (TABLE `t`) TABLE `cte`"}, {"with cte as (select 1) select 1 union with cte as (select 1) select * from cte;", false, ""}, {"with cte as (select 1) (select 1);", true, "WITH `cte` AS (SELECT 1) (SELECT 1)"}, @@ -6904,6 +6927,7 @@ func TestCTEBindings(t *testing.T) { {"with cte(a) as (select * from t) delete t from t, cte where t.a=cte.a;", true, "WITH `cte` (`a`) AS (SELECT * FROM `test`.`t`) DELETE `test`.`t` FROM (`test`.`t`) JOIN `cte` WHERE `t`.`a` = `cte`.`a`"}, {"WITH cte1 AS (SELECT * from t) SELECT * FROM (WITH cte2 AS (SELECT * from cte1) SELECT * FROM cte2 JOIN cte1) AS dt;", true, "WITH `cte1` AS (SELECT * FROM `test`.`t`) SELECT * FROM (WITH `cte2` AS (SELECT * FROM `cte1`) SELECT * FROM `cte2` JOIN `cte1`) AS `dt`"}, {"WITH cte AS (SELECT * from t) SELECT /*+ MAX_EXECUTION_TIME(1000) */ * FROM cte;", true, "WITH `cte` AS (SELECT * FROM `test`.`t`) SELECT /*+ MAX_EXECUTION_TIME(1000)*/ * FROM `cte`"}, + {"WITH cte AS (SELECT * from t) SELECT /*+ TIDB_KV_READ_TIMEOUT(1000) */ * FROM cte;", true, "WITH `cte` AS (SELECT * FROM `test`.`t`) SELECT /*+ TIDB_KV_READ_TIMEOUT(1000)*/ * FROM `cte`"}, {"with cte as (table t) table cte;", true, "WITH `cte` AS (TABLE `test`.`t`) TABLE `cte`"}, {"with cte as (select * from t) select 1 union with cte as (select * from t) select * from cte;", false, ""}, {"with cte as (select * from t) (select * from t);", true, "WITH `cte` AS (SELECT * FROM `test`.`t`) (SELECT * FROM `test`.`t`)"}, diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index defffec481895..ae84c041d0c1c 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -151,6 +151,17 @@ const ( HintSemiJoinRewrite = "semi_join_rewrite" // HintNoDecorrelate indicates a LogicalApply not to be decorrelated. HintNoDecorrelate = "no_decorrelate" + + // HintMemoryQuota sets the memory limit for a query + HintMemoryQuota = "memory_quota" + // HintUseToja is a hint to optimize `in (select ...)` subquery into `join` + HintUseToja = "use_toja" + // HintNoIndexMerge is a hint to disable index merge + HintNoIndexMerge = "no_index_merge" + // HintMaxExecutionTime specifies the max allowed execution time in milliseconds + HintMaxExecutionTime = "max_execution_time" + // HintTidbKvReadTimeout specifies timeout value for any readonly kv request in milliseconds + HintTidbKvReadTimeout = "tidb_kv_read_timeout" ) const ( diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 2e69205a2b9c6..ba5ed226819ef 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -638,6 +638,16 @@ func TestNthPlanHint(t *testing.T) { "Warning 1105 The parameter of nth_plan() is out of range")) } +func TestTiDBKvReadTimeoutHint(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int);") + tk.MustQuery("select /*+ tidb_kv_read_timeout(1) tidb_kv_read_timeout(2) */ * from t where a=1").Check(testkit.Rows()) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 TIDB_KV_READ_TIMEOUT() is defined more than once, only the last definition takes effect: TIDB_KV_READ_TIMEOUT(2)")) +} + func BenchmarkDecodePlan(b *testing.B) { store := testkit.CreateMockStore(b) tk := testkit.NewTestKit(b, store) diff --git a/planner/optimize.go b/planner/optimize.go index d5ee997057180..6d66467bab0a2 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -586,7 +586,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin } hintOffs := make(map[string]int, len(hints)) var forceNthPlan *ast.TableOptimizerHint - var memoryQuotaHintCnt, useToJAHintCnt, useCascadesHintCnt, noIndexMergeHintCnt, readReplicaHintCnt, maxExecutionTimeCnt, forceNthPlanCnt, straightJoinHintCnt int + var memoryQuotaHintCnt, useToJAHintCnt, useCascadesHintCnt, noIndexMergeHintCnt, readReplicaHintCnt, maxExecutionTimeCnt, tidbKvReadTimeoutCnt, forceNthPlanCnt, straightJoinHintCnt int setVars := make(map[string]string) setVarsOffs := make([]int, 0, len(hints)) for i, hint := range hints { @@ -609,6 +609,9 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin case "max_execution_time": hintOffs[hint.HintName.L] = i maxExecutionTimeCnt++ + case "tidb_kv_read_timeout": + hintOffs[hint.HintName.L] = i + tidbKvReadTimeoutCnt++ case "nth_plan": forceNthPlanCnt++ forceNthPlan = hint @@ -717,6 +720,16 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin stmtHints.HasMaxExecutionTime = true stmtHints.MaxExecutionTime = maxExecutionTime.HintData.(uint64) } + // Handle TIDB_KV_READ_TIMEOUT + if tidbKvReadTimeoutCnt != 0 { + tidbKvReadTimeout := hints[hintOffs["tidb_kv_read_timeout"]] + if tidbKvReadTimeoutCnt > 1 { + warn := errors.Errorf("TIDB_KV_READ_TIMEOUT() is defined more than once, only the last definition takes effect: TIDB_KV_READ_TIMEOUT(%v)", tidbKvReadTimeout.HintData.(uint64)) + warns = append(warns, warn) + } + stmtHints.HasTidbKvReadTimeout = true + stmtHints.TidbKvReadTimeout = tidbKvReadTimeout.HintData.(uint64) + } // Handle NTH_PLAN if forceNthPlanCnt != 0 { if forceNthPlanCnt > 1 { diff --git a/session/test/vars/vars_test.go b/session/test/vars/vars_test.go new file mode 100644 index 0000000000000..29e51ec787577 --- /dev/null +++ b/session/test/vars/vars_test.go @@ -0,0 +1,617 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vars + +import ( + "context" + "fmt" + "strconv" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" + tikv "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/txnkv/transaction" +) + +func TestKVVars(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_backoff_lock_fast = 1") + tk.MustExec("set @@tidb_backoff_weight = 100") + tk.MustExec("create table if not exists kvvars (a int key)") + tk.MustExec("insert into kvvars values (1)") + tk.MustExec("begin") + txn, err := tk.Session().Txn(false) + require.NoError(t, err) + vars := txn.GetVars().(*tikv.Variables) + require.Equal(t, 1, vars.BackoffLockFast) + require.Equal(t, 100, vars.BackOffWeight) + tk.MustExec("rollback") + tk.MustExec("set @@tidb_backoff_weight = 50") + tk.MustExec("set @@autocommit = 0") + tk.MustExec("select * from kvvars") + require.True(t, tk.Session().GetSessionVars().InTxn()) + txn, err = tk.Session().Txn(false) + require.NoError(t, err) + vars = txn.GetVars().(*tikv.Variables) + require.Equal(t, 50, vars.BackOffWeight) + + tk.MustExec("set @@autocommit = 1") + require.Nil(t, failpoint.Enable("tikvclient/probeSetVars", `return(true)`)) + tk.MustExec("select * from kvvars where a = 1") + require.Nil(t, failpoint.Disable("tikvclient/probeSetVars")) + require.True(t, transaction.SetSuccess.Load()) + transaction.SetSuccess.Store(false) +} + +func TestRemovedSysVars(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "bogus_var", Value: "acdc"}) + result := tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'bogus_var'") + result.Check(testkit.Rows("bogus_var acdc")) + result = tk.MustQuery("SELECT @@GLOBAL.bogus_var") + result.Check(testkit.Rows("acdc")) + tk.MustExec("SET GLOBAL bogus_var = 'newvalue'") + + // unregister + variable.UnregisterSysVar("bogus_var") + + result = tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'bogus_var'") + result.Check(testkit.Rows()) // empty + tk.MustContainErrMsg("SET GLOBAL bogus_var = 'newvalue'", "[variable:1193]Unknown system variable 'bogus_var'") + tk.MustContainErrMsg("SELECT @@GLOBAL.bogus_var", "[variable:1193]Unknown system variable 'bogus_var'") +} + +func TestTiKVSystemVars(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + result := tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'tidb_gc_enable'") // default is on from the sysvar + result.Check(testkit.Rows("tidb_gc_enable ON")) + result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_enable'") + result.Check(testkit.Rows()) // but no value in the table (yet) because the value has not been set and the GC has never been run + + // update will set a value in the table + tk.MustExec("SET GLOBAL tidb_gc_enable = 1") + result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_enable'") + result.Check(testkit.Rows("true")) + + tk.MustExec("UPDATE mysql.tidb SET variable_value = 'false' WHERE variable_name='tikv_gc_enable'") + result = tk.MustQuery("SELECT @@tidb_gc_enable;") + result.Check(testkit.Rows("0")) // reads from mysql.tidb value and changes to false + + tk.MustExec("SET GLOBAL tidb_gc_concurrency = -1") // sets auto concurrency and concurrency + result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_auto_concurrency'") + result.Check(testkit.Rows("true")) + result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_concurrency'") + result.Check(testkit.Rows("-1")) + + tk.MustExec("SET GLOBAL tidb_gc_concurrency = 5") // sets auto concurrency and concurrency + result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_auto_concurrency'") + result.Check(testkit.Rows("false")) + result = tk.MustQuery("SELECT variable_value FROM mysql.tidb WHERE variable_name = 'tikv_gc_concurrency'") + result.Check(testkit.Rows("5")) + + tk.MustExec("UPDATE mysql.tidb SET variable_value = 'true' WHERE variable_name='tikv_gc_auto_concurrency'") + result = tk.MustQuery("SELECT @@tidb_gc_concurrency;") + result.Check(testkit.Rows("-1")) // because auto_concurrency is turned on it takes precedence + + tk.MustExec("REPLACE INTO mysql.tidb (variable_value, variable_name) VALUES ('15m', 'tikv_gc_run_interval')") + result = tk.MustQuery("SELECT @@GLOBAL.tidb_gc_run_interval;") + result.Check(testkit.Rows("15m0s")) + result = tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'tidb_gc_run_interval'") + result.Check(testkit.Rows("tidb_gc_run_interval 15m0s")) + + tk.MustExec("SET GLOBAL tidb_gc_run_interval = '9m'") // too small + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_gc_run_interval value: '9m'")) + result = tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'tidb_gc_run_interval'") + result.Check(testkit.Rows("tidb_gc_run_interval 10m0s")) + + tk.MustExec("SET GLOBAL tidb_gc_run_interval = '700000000000ns'") // specified in ns, also valid + + _, err := tk.Exec("SET GLOBAL tidb_gc_run_interval = '11mins'") + require.Equal(t, "[variable:1232]Incorrect argument type to variable 'tidb_gc_run_interval'", err.Error()) +} + +func TestUpgradeSysvars(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session().(variable.GlobalVarAccessor) + + // Set the global var to a non-canonical form of the value + // i.e. implying that it was set from an earlier version of TiDB. + + tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('tidb_enable_noop_functions', '0')`) + domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache + v, err := se.GetGlobalSysVar("tidb_enable_noop_functions") + require.NoError(t, err) + require.Equal(t, "OFF", v) + + // Set the global var to "" which is the invalid version of this from TiDB 4.0.16 + // the err is quashed by the GetGlobalSysVar, and the default value is restored. + // This helps callers of GetGlobalSysVar(), which can't individually be expected + // to handle upgrade/downgrade issues correctly. + + tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('rpl_semi_sync_slave_enabled', '')`) + domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache + v, err = se.GetGlobalSysVar("rpl_semi_sync_slave_enabled") + require.NoError(t, err) + require.Equal(t, "OFF", v) // the default value is restored. + result := tk.MustQuery("SHOW VARIABLES LIKE 'rpl_semi_sync_slave_enabled'") + result.Check(testkit.Rows("rpl_semi_sync_slave_enabled OFF")) + + // Ensure variable out of range is converted to in range after upgrade. + // This further helps for https://github.com/pingcap/tidb/pull/28842 + + tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('tidb_executor_concurrency', '999')`) + domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache + v, err = se.GetGlobalSysVar("tidb_executor_concurrency") + require.NoError(t, err) + require.Equal(t, "256", v) // the max value is restored. + + // Handle the case of a completely bogus value from an earlier version of TiDB. + // This could be the case if an ENUM sysvar removes a value. + + tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('tidb_enable_noop_functions', 'SOMEVAL')`) + domain.GetDomain(tk.Session()).NotifyUpdateSysVarCache(true) // update cache + v, err = se.GetGlobalSysVar("tidb_enable_noop_functions") + require.NoError(t, err) + require.Equal(t, "OFF", v) // the default value is restored. +} + +func TestSetInstanceSysvarBySetGlobalSysVar(t *testing.T) { + varName := "tidb_general_log" + defaultValue := "OFF" // This is the default value for tidb_general_log + + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session().(variable.GlobalVarAccessor) + + // Get globalSysVar twice and get the same default value + v, err := se.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, defaultValue, v) + v, err = se.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, defaultValue, v) + + // session.GetGlobalSysVar would not get the value which session.SetGlobalSysVar writes, + // because SetGlobalSysVar calls SetGlobalFromHook, which uses TiDBGeneralLog's SetGlobal, + // but GetGlobalSysVar could not access TiDBGeneralLog's GetGlobal. + + // set to "1" + err = se.SetGlobalSysVar(context.Background(), varName, "ON") + require.NoError(t, err) + v, err = se.GetGlobalSysVar(varName) + tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("1")) + require.NoError(t, err) + require.Equal(t, defaultValue, v) + + // set back to "0" + err = se.SetGlobalSysVar(context.Background(), varName, defaultValue) + require.NoError(t, err) + v, err = se.GetGlobalSysVar(varName) + tk.MustQuery("select @@global.tidb_general_log").Check(testkit.Rows("0")) + require.NoError(t, err) + require.Equal(t, defaultValue, v) +} + +func TestEnableLegacyInstanceScope(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + + // enable 'switching' to SESSION variables + tk.MustExec("set tidb_enable_legacy_instance_scope = 1") + tk.MustExec("set tidb_general_log = 1") + tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_general_log will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope))) + require.True(t, tk.Session().GetSessionVars().EnableLegacyInstanceScope) + + // disable 'switching' to SESSION variables + tk.MustExec("set tidb_enable_legacy_instance_scope = 0") + tk.MustGetErrCode("set tidb_general_log = 1", errno.ErrGlobalVariable) + require.False(t, tk.Session().GetSessionVars().EnableLegacyInstanceScope) +} + +func TestSetPDClientDynamicOption(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 0.5;") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0.5")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 1;") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("1")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 1.5;") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("1.5")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 10;") + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("10")) + require.Error(t, tk.ExecToErr("set tidb_tso_client_batch_max_wait_time = 0;")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = -1;") + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '-1'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = -0.1;") + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '-0.1'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 10.1;") + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '10.1'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("10")) + tk.MustExec("set global tidb_tso_client_batch_max_wait_time = 11;") + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_tso_client_batch_max_wait_time value: '11'")) + tk.MustQuery("select @@tidb_tso_client_batch_max_wait_time;").Check(testkit.Rows("10")) + + tk.MustQuery("select @@tidb_enable_tso_follower_proxy;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_enable_tso_follower_proxy = on;") + tk.MustQuery("select @@tidb_enable_tso_follower_proxy;").Check(testkit.Rows("1")) + tk.MustExec("set global tidb_enable_tso_follower_proxy = off;") + tk.MustQuery("select @@tidb_enable_tso_follower_proxy;").Check(testkit.Rows("0")) + require.Error(t, tk.ExecToErr("set tidb_tso_client_batch_max_wait_time = 0;")) +} + +func TestCastTimeToDate(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set time_zone = '-8:00'") + date := time.Now().In(time.FixedZone("", -8*int(time.Hour/time.Second))) + tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format(time.DateOnly))) + + tk.MustExec("set time_zone = '+08:00'") + date = time.Now().In(time.FixedZone("", 8*int(time.Hour/time.Second))) + tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format(time.DateOnly))) +} + +func TestSetGlobalTZ(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set time_zone = '+08:00'") + tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) + + tk.MustExec("set global time_zone = '+00:00'") + + tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +00:00")) +} +func TestSetVarHint(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("sql_mode", mysql.DefaultSQLMode)) + tk.MustQuery("SELECT /*+ SET_VAR(sql_mode=ALLOW_INVALID_DATES) */ @@sql_mode;").Check(testkit.Rows("ALLOW_INVALID_DATES")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@sql_mode;").Check(testkit.Rows(mysql.DefaultSQLMode)) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tmp_table_size", "16777216")) + tk.MustQuery("SELECT /*+ SET_VAR(tmp_table_size=1024) */ @@tmp_table_size;").Check(testkit.Rows("1024")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@tmp_table_size;").Check(testkit.Rows("16777216")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("range_alloc_block_size", "4096")) + tk.MustQuery("SELECT /*+ SET_VAR(range_alloc_block_size=4294967295) */ @@range_alloc_block_size;").Check(testkit.Rows("4294967295")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@range_alloc_block_size;").Check(testkit.Rows("4096")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_execution_time", "0")) + tk.MustQuery("SELECT /*+ SET_VAR(max_execution_time=1) */ @@max_execution_time;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_execution_time;").Check(testkit.Rows("0")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tidb_kv_read_timeout", "0")) + tk.MustQuery("SELECT /*+ SET_VAR(tidb_kv_read_timeout=10) */ @@tidb_kv_read_timeout;").Check(testkit.Rows("10")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@tidb_kv_read_timeout;").Check(testkit.Rows("0")) + + tk.MustExec("set @@tidb_kv_read_timeout = 5") + tk.MustQuery("SELECT /*+ tidb_kv_read_timeout(1) */ @@tidb_kv_read_timeout;").Check(testkit.Rows("5")) + require.Equal(t, tk.Session().GetSessionVars().GetTidbKvReadTimeout(), uint64(1)) + tk.MustQuery("SELECT @@tidb_kv_read_timeout;").Check(testkit.Rows("5")) + require.Equal(t, tk.Session().GetSessionVars().GetTidbKvReadTimeout(), uint64(5)) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("time_zone", "SYSTEM")) + tk.MustQuery("SELECT /*+ SET_VAR(time_zone='+12:00') */ @@time_zone;").Check(testkit.Rows("+12:00")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@time_zone;").Check(testkit.Rows("SYSTEM")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("join_buffer_size", "262144")) + tk.MustQuery("SELECT /*+ SET_VAR(join_buffer_size=128) */ @@join_buffer_size;").Check(testkit.Rows("128")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@join_buffer_size;").Check(testkit.Rows("262144")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_length_for_sort_data", "1024")) + tk.MustQuery("SELECT /*+ SET_VAR(max_length_for_sort_data=4) */ @@max_length_for_sort_data;").Check(testkit.Rows("4")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_length_for_sort_data;").Check(testkit.Rows("1024")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_error_count", "64")) + tk.MustQuery("SELECT /*+ SET_VAR(max_error_count=0) */ @@max_error_count;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_error_count;").Check(testkit.Rows("64")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("sql_buffer_result", "OFF")) + tk.MustQuery("SELECT /*+ SET_VAR(sql_buffer_result=ON) */ @@sql_buffer_result;").Check(testkit.Rows("ON")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@sql_buffer_result;").Check(testkit.Rows("OFF")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_heap_table_size", "16777216")) + tk.MustQuery("SELECT /*+ SET_VAR(max_heap_table_size=16384) */ @@max_heap_table_size;").Check(testkit.Rows("16384")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_heap_table_size;").Check(testkit.Rows("16777216")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tmp_table_size", "16777216")) + tk.MustQuery("SELECT /*+ SET_VAR(tmp_table_size=16384) */ @@tmp_table_size;").Check(testkit.Rows("16384")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@tmp_table_size;").Check(testkit.Rows("16777216")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("div_precision_increment", "4")) + tk.MustQuery("SELECT /*+ SET_VAR(div_precision_increment=0) */ @@div_precision_increment;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@div_precision_increment;").Check(testkit.Rows("4")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("sql_auto_is_null", "OFF")) + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tidb_enable_noop_functions", "ON")) + tk.MustQuery("SELECT /*+ SET_VAR(sql_auto_is_null=1) */ @@sql_auto_is_null;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + require.NoError(t, tk.Session().GetSessionVars().SetSystemVarWithoutValidation("tidb_enable_noop_functions", "OFF")) + tk.MustQuery("SELECT @@sql_auto_is_null;").Check(testkit.Rows("0")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("sort_buffer_size", "262144")) + tk.MustQuery("SELECT /*+ SET_VAR(sort_buffer_size=32768) */ @@sort_buffer_size;").Check(testkit.Rows("32768")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@sort_buffer_size;").Check(testkit.Rows("262144")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_join_size", "18446744073709551615")) + tk.MustQuery("SELECT /*+ SET_VAR(max_join_size=1) */ @@max_join_size;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_join_size;").Check(testkit.Rows("18446744073709551615")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_seeks_for_key", "18446744073709551615")) + tk.MustQuery("SELECT /*+ SET_VAR(max_seeks_for_key=1) */ @@max_seeks_for_key;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_seeks_for_key;").Check(testkit.Rows("18446744073709551615")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_sort_length", "1024")) + tk.MustQuery("SELECT /*+ SET_VAR(max_sort_length=4) */ @@max_sort_length;").Check(testkit.Rows("4")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_sort_length;").Check(testkit.Rows("1024")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("bulk_insert_buffer_size", "8388608")) + tk.MustQuery("SELECT /*+ SET_VAR(bulk_insert_buffer_size=0) */ @@bulk_insert_buffer_size;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@bulk_insert_buffer_size;").Check(testkit.Rows("8388608")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("sql_big_selects", "1")) + tk.MustQuery("SELECT /*+ SET_VAR(sql_big_selects=0) */ @@sql_big_selects;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@sql_big_selects;").Check(testkit.Rows("1")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("read_rnd_buffer_size", "262144")) + tk.MustQuery("SELECT /*+ SET_VAR(read_rnd_buffer_size=1) */ @@read_rnd_buffer_size;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@read_rnd_buffer_size;").Check(testkit.Rows("262144")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("unique_checks", "1")) + tk.MustQuery("SELECT /*+ SET_VAR(unique_checks=0) */ @@unique_checks;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@unique_checks;").Check(testkit.Rows("1")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("read_buffer_size", "131072")) + tk.MustQuery("SELECT /*+ SET_VAR(read_buffer_size=8192) */ @@read_buffer_size;").Check(testkit.Rows("8192")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@read_buffer_size;").Check(testkit.Rows("131072")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("default_tmp_storage_engine", "InnoDB")) + tk.MustQuery("SELECT /*+ SET_VAR(default_tmp_storage_engine='CSV') */ @@default_tmp_storage_engine;").Check(testkit.Rows("CSV")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@default_tmp_storage_engine;").Check(testkit.Rows("InnoDB")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("optimizer_search_depth", "62")) + tk.MustQuery("SELECT /*+ SET_VAR(optimizer_search_depth=1) */ @@optimizer_search_depth;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@optimizer_search_depth;").Check(testkit.Rows("62")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("max_points_in_geometry", "65536")) + tk.MustQuery("SELECT /*+ SET_VAR(max_points_in_geometry=3) */ @@max_points_in_geometry;").Check(testkit.Rows("3")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@max_points_in_geometry;").Check(testkit.Rows("65536")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("updatable_views_with_limit", "YES")) + tk.MustQuery("SELECT /*+ SET_VAR(updatable_views_with_limit=0) */ @@updatable_views_with_limit;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@updatable_views_with_limit;").Check(testkit.Rows("YES")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("optimizer_prune_level", "1")) + tk.MustQuery("SELECT /*+ SET_VAR(optimizer_prune_level=0) */ @@optimizer_prune_level;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@optimizer_prune_level;").Check(testkit.Rows("1")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("group_concat_max_len", "1024")) + tk.MustQuery("SELECT /*+ SET_VAR(group_concat_max_len=4) */ @@group_concat_max_len;").Check(testkit.Rows("4")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@group_concat_max_len;").Check(testkit.Rows("1024")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("eq_range_index_dive_limit", "200")) + tk.MustQuery("SELECT /*+ SET_VAR(eq_range_index_dive_limit=0) */ @@eq_range_index_dive_limit;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@eq_range_index_dive_limit;").Check(testkit.Rows("200")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("sql_safe_updates", "0")) + tk.MustQuery("SELECT /*+ SET_VAR(sql_safe_updates=1) */ @@sql_safe_updates;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@sql_safe_updates;").Check(testkit.Rows("0")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("end_markers_in_json", "0")) + tk.MustQuery("SELECT /*+ SET_VAR(end_markers_in_json=1) */ @@end_markers_in_json;").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@end_markers_in_json;").Check(testkit.Rows("0")) + + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("windowing_use_high_precision", "ON")) + tk.MustQuery("SELECT /*+ SET_VAR(windowing_use_high_precision=OFF) */ @@windowing_use_high_precision;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + tk.MustQuery("SELECT @@windowing_use_high_precision;").Check(testkit.Rows("1")) + + tk.MustExec("SELECT /*+ SET_VAR(sql_safe_updates = 1) SET_VAR(max_heap_table_size = 1G) */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) + + tk.MustExec("SELECT /*+ SET_VAR(collation_server = 'utf8') */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[planner:3637]Variable 'collation_server' cannot be set using SET_VAR hint.") + + tk.MustExec("SELECT /*+ SET_VAR(max_size = 1G) */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[planner:3128]Unresolved name 'max_size' for SET_VAR hint") + + tk.MustExec("SELECT /*+ SET_VAR(group_concat_max_len = 1024) SET_VAR(group_concat_max_len = 2048) */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[planner:3126]Hint SET_VAR(group_concat_max_len=2048) is ignored as conflicting/duplicated.") +} + +func TestGlobalVarAccessor(t *testing.T) { + varName := "max_allowed_packet" + varValue := strconv.FormatUint(variable.DefMaxAllowedPacket, 10) // This is the default value for max_allowed_packet + + // The value of max_allowed_packet should be a multiple of 1024, + // so the setting of varValue1 and varValue2 would be truncated to varValue0 + varValue0 := "4194304" + varValue1 := "4194305" + varValue2 := "4194306" + + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + se := tk.Session().(variable.GlobalVarAccessor) + // Get globalSysVar twice and get the same value + v, err := se.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, varValue, v) + v, err = se.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, varValue, v) + // Set global var to another value + err = se.SetGlobalSysVar(context.Background(), varName, varValue1) + require.NoError(t, err) + v, err = se.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, varValue0, v) + require.NoError(t, tk.Session().CommitTxn(context.TODO())) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + se1 := tk1.Session().(variable.GlobalVarAccessor) + v, err = se1.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, varValue0, v) + err = se1.SetGlobalSysVar(context.Background(), varName, varValue2) + require.NoError(t, err) + v, err = se1.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, varValue0, v) + require.NoError(t, tk1.Session().CommitTxn(context.TODO())) + + // Make sure the change is visible to any client that accesses that global variable. + v, err = se.GetGlobalSysVar(varName) + require.NoError(t, err) + require.Equal(t, varValue0, v) + + // For issue 10955, make sure the new session load `max_execution_time` into sessionVars. + tk1.MustExec("set @@global.max_execution_time = 100") + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + require.Equal(t, uint64(100), tk2.Session().GetSessionVars().MaxExecutionTime) + tk1.MustExec("set @@global.max_execution_time = 0") + + result := tk.MustQuery("show global variables where variable_name='sql_select_limit';") + result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) + result = tk.MustQuery("show session variables where variable_name='sql_select_limit';") + result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) + tk.MustExec("set session sql_select_limit=100000000000;") + result = tk.MustQuery("show global variables where variable_name='sql_select_limit';") + result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) + result = tk.MustQuery("show session variables where variable_name='sql_select_limit';") + result.Check(testkit.Rows("sql_select_limit 100000000000")) + tk.MustExec("set @@global.sql_select_limit = 1") + result = tk.MustQuery("show global variables where variable_name='sql_select_limit';") + result.Check(testkit.Rows("sql_select_limit 1")) + tk.MustExec("set @@global.sql_select_limit = default") + result = tk.MustQuery("show global variables where variable_name='sql_select_limit';") + result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) + + result = tk.MustQuery("select @@global.autocommit;") + result.Check(testkit.Rows("1")) + result = tk.MustQuery("select @@autocommit;") + result.Check(testkit.Rows("1")) + tk.MustExec("set @@global.autocommit = 0;") + result = tk.MustQuery("select @@global.autocommit;") + result.Check(testkit.Rows("0")) + result = tk.MustQuery("select @@autocommit;") + result.Check(testkit.Rows("1")) + tk.MustExec("set @@global.autocommit=1") + + err = tk.ExecToErr("set global time_zone = 'timezone'") + require.Error(t, err) + require.True(t, terror.ErrorEqual(err, variable.ErrUnknownTimeZone)) +} + +func TestGetSysVariables(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // Test ScopeSession + tk.MustExec("select @@warning_count") + tk.MustExec("select @@session.warning_count") + tk.MustExec("select @@local.warning_count") + err := tk.ExecToErr("select @@global.warning_count") + require.True(t, terror.ErrorEqual(err, variable.ErrIncorrectScope), fmt.Sprintf("err %v", err)) + + // Test ScopeGlobal + tk.MustExec("select @@max_connections") + tk.MustExec("select @@global.max_connections") + tk.MustGetErrMsg("select @@session.max_connections", "[variable:1238]Variable 'max_connections' is a GLOBAL variable") + tk.MustGetErrMsg("select @@local.max_connections", "[variable:1238]Variable 'max_connections' is a GLOBAL variable") + + // Test ScopeNone + tk.MustExec("select @@performance_schema_max_mutex_classes") + tk.MustExec("select @@global.performance_schema_max_mutex_classes") + // For issue 19524, test + tk.MustExec("select @@session.performance_schema_max_mutex_classes") + tk.MustExec("select @@local.performance_schema_max_mutex_classes") + tk.MustGetErrMsg("select @@global.last_insert_id", "[variable:1238]Variable 'last_insert_id' is a SESSION variable") +} diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index d8e0bc7cd5fb7..6e50bbd3502e5 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -390,6 +390,7 @@ type StmtHints struct { // Hint Information MemQuotaQuery int64 MaxExecutionTime uint64 + TidbKvReadTimeout uint64 ReplicaRead byte AllowInSubqToJoinAndAgg bool NoIndexMergeHint bool @@ -405,6 +406,7 @@ type StmtHints struct { HasMemQuotaHint bool HasReplicaReadHint bool HasMaxExecutionTime bool + HasTidbKvReadTimeout bool HasEnableCascadesPlannerHint bool SetVars map[string]string diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 31e012d03ad26..17959ef810835 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -974,6 +974,10 @@ type SessionVars struct { // See https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_max_execution_time MaxExecutionTime uint64 + // TidbKvReadTimeout is the timeout for readonly kv request in milliseconds, 0 means using default value + // See https://github.com/pingcap/tidb/blob/7105505a78fc886c33258caa5813baf197b15247/docs/design/2023-06-30-configurable-kv-timeout.md?plain=1#L14-L15 + TidbKvReadTimeout uint64 + // Killed is a flag to indicate that this query is killed. Killed uint32 @@ -3177,3 +3181,11 @@ func (s *SessionVars) GetRelatedTableForMDL() *sync.Map { func (s *SessionVars) EnableForceInlineCTE() bool { return s.enableForceInlineCTE } + +// GetTidbKvReadTimeout returns readonly kv request timeout, prefer query hint over session variable +func (s *SessionVars) GetTidbKvReadTimeout() uint64 { + if s.StmtCtx.HasTidbKvReadTimeout { + return s.StmtCtx.TidbKvReadTimeout + } + return s.TidbKvReadTimeout +} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index c095d7e8a3d1c..b45c13e90ef45 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1224,6 +1224,11 @@ var defaultSysVars = []*SysVar{ s.MaxExecutionTime = uint64(timeoutMS) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TidbKvReadTimeout, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { + timeoutMS := tidbOptPositiveInt32(val, 0) + s.TidbKvReadTimeout = uint64(timeoutMS) + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationServer, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { @@ -2584,6 +2589,8 @@ const ( TxnIsolationOneShot = "tx_isolation_one_shot" // MaxExecutionTime is the name of the 'max_execution_time' system variable. MaxExecutionTime = "max_execution_time" + // TidbKvReadTimeout is the name of the 'tidb_kv_read_timeout' system variable. + TidbKvReadTimeout = "tidb_kv_read_timeout" // ReadOnly is the name of the 'read_only' system variable. ReadOnly = "read_only" // DefaultAuthPlugin is the name of 'default_authentication_plugin' system variable. diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index cdd48a789b9b5..04bf4bf6ad173 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -694,6 +694,10 @@ func TestStmtVars(t *testing.T) { require.Equal(t, "[variable:1232]Incorrect argument type to variable 'max_execution_time'", err.Error()) err = vars.SetStmtVar(MaxExecutionTime, "100") require.NoError(t, err) + err = vars.SetStmtVar(TidbKvReadTimeout, "ACDC") + require.Equal(t, "[variable:1232]Incorrect argument type to variable 'tidb_kv_read_timeout'", err.Error()) + err = vars.SetStmtVar(TidbKvReadTimeout, "100") + require.NoError(t, err) } func TestSessionStatesSystemVar(t *testing.T) { diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 26229de333904..1923b04ed0bd9 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -154,7 +154,7 @@ func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars ) buildTaskFunc := func(ranges []kv.KeyRange) error { keyRanges := NewKeyRanges(ranges) - tasksFromRanges, err := buildCopTasks(bo, c.store.GetRegionCache(), keyRanges, req, eventCb) + tasksFromRanges, err := buildCopTasks(bo, c.store.GetRegionCache(), keyRanges, req, eventCb, false) if err != nil { return err } @@ -269,6 +269,9 @@ type copTask struct { RowCountHint int // used for extra concurrency of small tasks, -1 for unknown row count batchTaskList map[uint64]*batchedCopTask meetLockFallback bool + + // timeout value for one kv readonly reqeust + tidbKvReadTimeout uint64 } type batchedCopTask struct { @@ -303,7 +306,7 @@ func (r *copTask) ToPBBatchTasks() []*coprocessor.StoreBatchTask { // rangesPerTask limits the length of the ranges slice sent in one copTask. const rangesPerTask = 25000 -func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv.Request, eventCb trxevents.EventCallback) ([]*copTask, error) { +func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv.Request, eventCb trxevents.EventCallback, ignoreTiDBKVReadTimeout bool) ([]*copTask, error) { start := time.Now() cmdType := tikvrpc.CmdCop if req.StoreType == kv.TiDB { @@ -412,6 +415,10 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv } else { tasks = append(tasks, task) } + if !ignoreTiDBKVReadTimeout { + task.tidbKvReadTimeout = req.TidbKvReadTimeout + } + i = nextI if req.Paging.Enable { if req.LimitSize != 0 && req.LimitSize < pagingSize { @@ -1020,6 +1027,10 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch if worker.req.ResourceGroupTagger != nil { worker.req.ResourceGroupTagger(req) } + timeout := tikv.ReadTimeoutMedium + if task.tidbKvReadTimeout > 0 { + timeout = time.Duration(task.tidbKvReadTimeout) * time.Millisecond + } req.StoreTp = getEndPointType(task.storeType) startTime := time.Now() if worker.kvclient.Stats == nil { @@ -1038,7 +1049,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch if len(worker.req.MatchStoreLabels) > 0 { ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } - resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo.TiKVBackoffer(), req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) + resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo.TiKVBackoffer(), req, task.region, timeout, getEndPointType(task.storeType), task.storeAddr, ops...) err = derr.ToTiDBErr(err) if err != nil { if task.storeType == kv.TiDB { @@ -1170,7 +1181,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R return nil, errors.Trace(err) } // We may meet RegionError at the first packet, but not during visiting the stream. - remains, err := buildCopTasks(bo, worker.store.GetRegionCache(), task.ranges, worker.req, task.eventCb) + remains, err := buildCopTasks(bo, worker.store.GetRegionCache(), task.ranges, worker.req, task.eventCb, true) if err != nil { return remains, err } @@ -1317,7 +1328,7 @@ func (worker *copIteratorWorker) handleBatchCopResponse(bo *Backoffer, rpcCtx *t if err := bo.Backoff(tikv.BoRegionMiss(), errors.New(errStr)); err != nil { return nil, errors.Trace(err) } - remains, err := buildCopTasks(bo, worker.store.GetRegionCache(), task.ranges, worker.req, task.eventCb) + remains, err := buildCopTasks(bo, worker.store.GetRegionCache(), task.ranges, worker.req, task.eventCb, true) if err != nil { return nil, err } diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index 5cb49648355e7..456a7a053f99f 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -50,49 +50,49 @@ func TestBuildTasksWithoutBuckets(t *testing.T) { req := &kv.Request{} flashReq := &kv.Request{} flashReq.StoreType = kv.TiFlash - tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c"), req, nil) + tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "c") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "c"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "c"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "c") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[1], 0, "g", "n") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[1], 0, "g", "n") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("m", "n"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("m", "n"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[1], 0, "m", "n") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("m", "n"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("m", "n"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[1], 0, "m", "n") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "k"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "k"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 2) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "g") taskEqual(t, tasks[1], regionIDs[1], 0, "g", "k") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "k"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "k"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 2) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "g") taskEqual(t, tasks[1], regionIDs[1], 0, "g", "k") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "x"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "x"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 4) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "g") @@ -100,7 +100,7 @@ func TestBuildTasksWithoutBuckets(t *testing.T) { taskEqual(t, tasks[2], regionIDs[2], 0, "n", "t") taskEqual(t, tasks[3], regionIDs[3], 0, "t", "x") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "x"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "x"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 4) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "g") @@ -108,45 +108,45 @@ func TestBuildTasksWithoutBuckets(t *testing.T) { taskEqual(t, tasks[2], regionIDs[2], 0, "n", "t") taskEqual(t, tasks[3], regionIDs[3], 0, "t", "x") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "b", "c"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "b", "c"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "b", "b", "c") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "b", "c"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "b", "c"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "b", "b", "c") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "e", "f"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "e", "f"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "b", "e", "f") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "e", "f"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "b", "e", "f"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "b", "e", "f") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n", "o", "p"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n", "o", "p"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 2) taskEqual(t, tasks[0], regionIDs[1], 0, "g", "n") taskEqual(t, tasks[1], regionIDs[2], 0, "o", "p") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n", "o", "p"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("g", "n", "o", "p"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 2) taskEqual(t, tasks[0], regionIDs[1], 0, "g", "n") taskEqual(t, tasks[1], regionIDs[2], 0, "o", "p") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("h", "k", "m", "p"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("h", "k", "m", "p"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 2) taskEqual(t, tasks[0], regionIDs[1], 0, "h", "k", "m", "n") taskEqual(t, tasks[1], regionIDs[2], 0, "n", "p") - tasks, err = buildCopTasks(bo, cache, buildCopRanges("h", "k", "m", "p"), flashReq, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("h", "k", "m", "p"), flashReq, nil, false) require.NoError(t, err) require.Len(t, tasks, 2) taskEqual(t, tasks[0], regionIDs[1], 0, "h", "k", "m", "n") @@ -191,7 +191,7 @@ func TestBuildTasksByBuckets(t *testing.T) { } for _, regionRange := range regionRanges { regionID, ranges := regionRange.regionID, regionRange.ranges - tasks, err := buildCopTasks(bo, cache, buildCopRanges(ranges...), req, nil) + tasks, err := buildCopTasks(bo, cache, buildCopRanges(ranges...), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(ranges)/2) for i, task := range tasks { @@ -204,7 +204,7 @@ func TestBuildTasksByBuckets(t *testing.T) { for _, regionRange := range regionRanges { allRanges = append(allRanges, regionRange.ranges...) } - tasks, err := buildCopTasks(bo, cache, buildCopRanges(allRanges...), req, nil) + tasks, err := buildCopTasks(bo, cache, buildCopRanges(allRanges...), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(allRanges)/2) taskIdx := 0 @@ -230,7 +230,7 @@ func TestBuildTasksByBuckets(t *testing.T) { "h", "i", "j", "k", "k", "l", "m", "n", } - tasks, err = buildCopTasks(bo, cache, buildCopRanges(keyRanges...), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges(keyRanges...), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(keyRanges)/4) for i, task := range tasks { @@ -251,7 +251,7 @@ func TestBuildTasksByBuckets(t *testing.T) { {"c", "d", "e", "g"}, {"g", "h", "i", "j"}, } - tasks, err = buildCopTasks(bo, cache, buildCopRanges(keyRanges...), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges(keyRanges...), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(expectedTaskRanges)) for i, task := range tasks { @@ -277,7 +277,7 @@ func TestBuildTasksByBuckets(t *testing.T) { cluster.SplitRegionBuckets(regionIDs[1], [][]byte{{'n'}, {'q'}, {'r'}, {'t'}, {'u'}, {'v'}, {'x'}}, regionIDs[1]) cache = NewRegionCache(tikv.NewRegionCache(pdCli)) defer cache.Close() - tasks, err = buildCopTasks(bo, cache, buildCopRanges("n", "o", "p", "q", "s", "w"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("n", "o", "p", "q", "s", "w"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(expectedTaskRanges)) for i, task := range tasks { @@ -301,7 +301,7 @@ func TestBuildTasksByBuckets(t *testing.T) { cluster.SplitRegionBuckets(regionIDs[1], [][]byte{{'q'}, {'s'}, {'u'}}, regionIDs[1]) cache = NewRegionCache(tikv.NewRegionCache(pdCli)) defer cache.Close() - tasks, err = buildCopTasks(bo, cache, buildCopRanges("n", "o", "p", "s", "t", "v", "w", "x"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("n", "o", "p", "s", "t", "v", "w", "x"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(expectedTaskRanges)) for i, task := range tasks { @@ -321,7 +321,7 @@ func TestBuildTasksByBuckets(t *testing.T) { cluster.SplitRegionBuckets(regionIDs[1], [][]byte{{'g'}, {'t'}, {'z'}}, regionIDs[1]) cache = NewRegionCache(tikv.NewRegionCache(pdCli)) defer cache.Close() - tasks, err = buildCopTasks(bo, cache, buildCopRanges("o", "p", "u", "w"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("o", "p", "u", "w"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(expectedTaskRanges)) for i, task := range tasks { @@ -343,7 +343,7 @@ func TestBuildTasksByBuckets(t *testing.T) { cluster.SplitRegionBuckets(regionIDs[1], [][]byte{{'n'}, {'q'}, {'r'}, {'x'}}, regionIDs[1]) cache = NewRegionCache(tikv.NewRegionCache(pdCli)) defer cache.Close() - tasks, err = buildCopTasks(bo, cache, buildCopRanges("n", "x"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("n", "x"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, len(expectedTaskRanges)) for i, task := range tasks { @@ -437,7 +437,7 @@ func TestRebuild(t *testing.T) { bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} - tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "z"), req, nil) + tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "z"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 2) taskEqual(t, tasks[0], regionIDs[0], 0, "a", "m") @@ -451,7 +451,7 @@ func TestRebuild(t *testing.T) { cache.InvalidateCachedRegion(tasks[1].region) req.Desc = true - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "z"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "z"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 3) taskEqual(t, tasks[2], regionIDs[0], 0, "a", "m") @@ -506,7 +506,7 @@ func TestBuildPagingTasks(t *testing.T) { req.Paging.MinPagingSize = paging.MinPagingSize flashReq := &kv.Request{} flashReq.StoreType = kv.TiFlash - tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c"), req, nil) + tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) require.Len(t, tasks, 1) @@ -537,7 +537,7 @@ func TestBuildPagingTasksDisablePagingForSmallLimit(t *testing.T) { req.Paging.Enable = true req.Paging.MinPagingSize = paging.MinPagingSize req.LimitSize = 1 - tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c"), req, nil) + tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c"), req, nil, false) require.NoError(t, err) require.Len(t, tasks, 1) require.Len(t, tasks, 1) @@ -711,7 +711,7 @@ func TestBuildCopTasksWithRowCountHint(t *testing.T) { bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} req.FixedRowCountHint = []int{1, 1, 3, CopSmallTaskRow} - tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c", "d", "e", "h", "x", "y", "z"), req, nil) + tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "c", "d", "e", "h", "x", "y", "z"), req, nil, false) require.Nil(t, err) require.Equal(t, len(tasks), 4) // task[0] ["a"-"c", "d"-"e"] @@ -726,7 +726,7 @@ func TestBuildCopTasksWithRowCountHint(t *testing.T) { require.Equal(t, conc, 1) req.FixedRowCountHint = []int{1, 1, 3, 3} - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "c", "d", "e", "h", "x", "y", "z"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "c", "d", "e", "h", "x", "y", "z"), req, nil, false) require.Nil(t, err) require.Equal(t, len(tasks), 4) // task[0] ["a"-"c", "d"-"e"] @@ -742,7 +742,7 @@ func TestBuildCopTasksWithRowCountHint(t *testing.T) { // cross-region long range req.FixedRowCountHint = []int{10} - tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "z"), req, nil) + tasks, err = buildCopTasks(bo, cache, buildCopRanges("a", "z"), req, nil, false) require.Nil(t, err) require.Equal(t, len(tasks), 4) // task[0] ["a"-"g"] diff --git a/store/driver/snap_interceptor_test.go b/store/driver/snap_interceptor_test.go index 1f278c0690942..33158c0edd108 100644 --- a/store/driver/snap_interceptor_test.go +++ b/store/driver/snap_interceptor_test.go @@ -210,6 +210,8 @@ func TestSnapshotWitInterceptor(t *testing.T) { require.Equal(t, "MockErrOnIterReverse", err.Error()) require.Nil(t, iter) require.Equal(t, []interface{}{"OnIterReverse", kv.Key{}}, mockInterceptor.spy) + + snap.SetOption(kv.TidbKvReadTimeout, uint64(10)) } func checkIter(t *testing.T, iter kv.Iterator, expected [][]interface{}) { diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 66c82f86eaedb..1fb4217036a15 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -16,6 +16,7 @@ package txn import ( "context" + "time" "unsafe" "github.com/pingcap/kvproto/pkg/metapb" @@ -135,6 +136,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { if size > 0 { s.KVSnapshot.SetScanBatchSize(size) } + case kv.TidbKvReadTimeout: + s.KVSnapshot.SetKVReadTimeout(time.Duration(val.(uint64) * uint64(time.Millisecond))) } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 8013c4b1cad63..dc88f14e09501 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -17,6 +17,7 @@ package txn import ( "context" "sync/atomic" + "time" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" @@ -266,6 +267,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetReplicaReadAdjuster(val.(txnkv.ReplicaReadAdjuster)) case kv.TxnSource: txn.KVTxn.SetTxnSource(val.(uint64)) + case kv.TidbKvReadTimeout: + txn.KVTxn.GetSnapshot().SetKVReadTimeout(time.Duration(val.(uint64) * uint64(time.Millisecond))) } } diff --git a/store/mockstore/unistore/rpc.go b/store/mockstore/unistore/rpc.go index b0e88e533582d..44a44e5985751 100644 --- a/store/mockstore/unistore/rpc.go +++ b/store/mockstore/unistore/rpc.go @@ -82,6 +82,11 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R } } }) + failpoint.Inject("unistoreRPCDeadlineExceeded", func(val failpoint.Value) { + if val.(bool) && timeout < time.Second { + failpoint.Return(tikvrpc.GenRegionErrorResp(req, &errorpb.Error{Message: "Deadline is exceeded"})) + } + }) select { case <-ctx.Done(): diff --git a/tests/realtikvtest/sessiontest/session_fail_test.go b/tests/realtikvtest/sessiontest/session_fail_test.go index a3df51be821c0..10af71ee421bf 100644 --- a/tests/realtikvtest/sessiontest/session_fail_test.go +++ b/tests/realtikvtest/sessiontest/session_fail_test.go @@ -16,6 +16,8 @@ package sessiontest import ( "context" + "fmt" + "strconv" "testing" "github.com/pingcap/failpoint" @@ -203,3 +205,82 @@ func TestKill(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("kill connection_id();") } + +func TestTidbKvReadTimeout(t *testing.T) { + if !*realtikvtest.WithRealTiKV { + t.Skip("skip test since it's only work for tikv") + } + store := realtikvtest.CreateMockStoreAndSetup(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int primary key, b int)") + + rows := tk.MustQuery("select count(*) from information_schema.cluster_info where `type`='tikv';").Rows() + require.Len(t, rows, 1) + tikvCount, err := strconv.Atoi(rows[0][0].(string)) + require.NoError(t, err) + if tikvCount < 3 { + t.Skip("skip test since it's only work for tikv with at least 3 node") + } + + require.NoError(t, failpoint.Enable("tikvclient/mockBatchClientSendDelay", "return(100)")) + defer func() { + require.NoError(t, failpoint.Disable("tikvclient/mockBatchClientSendDelay")) + }() + tk.MustExec("set @stale_read_ts_var=now(6);") + + // Test for point_get request + rows = tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t where a = 1").Rows() + require.Len(t, rows, 1) + explain := fmt.Sprintf("%v", rows[0]) + // num_rpc is 4 because there are 3 replica, and first try all 3 replicas with specified timeout will failed, then try again with default timeout will success. + require.Regexp(t, ".*Point_Get.* Get:{num_rpc:4, total_time:.*", explain) + + // Test for batch_point_get request + rows = tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t where a in (1,2)").Rows() + require.Len(t, rows, 1) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*Batch_Point_Get.* BatchGet:{num_rpc:4, total_time:.*", explain) + + // Test for cop request + rows = tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 4.*", explain) + + // Test for stale read. + tk.MustExec("insert into t values (1,1), (2,2);") + tk.MustExec("set @@tidb_replica_read='closest-replicas';") + rows = tk.MustQuery("explain analyze select /*+ tidb_kv_read_timeout(1) */ * from t as of timestamp(@stale_read_ts_var) where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: (3|4).*", explain) + + // Test for tidb_kv_read_timeout session variable. + tk.MustExec("set @@tidb_kv_read_timeout=1;") + // Test for point_get request + rows = tk.MustQuery("explain analyze select * from t where a = 1").Rows() + require.Len(t, rows, 1) + explain = fmt.Sprintf("%v", rows[0]) + // num_rpc is 4 because there are 3 replica, and first try all 3 replicas with specified timeout will failed, then try again with default timeout will success. + require.Regexp(t, ".*Point_Get.* Get:{num_rpc:4, total_time:.*", explain) + + // Test for batch_point_get request + rows = tk.MustQuery("explain analyze select * from t where a in (1,2)").Rows() + require.Len(t, rows, 1) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*Batch_Point_Get.* BatchGet:{num_rpc:4, total_time:.*", explain) + + // Test for cop request + rows = tk.MustQuery("explain analyze select * from t where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 4.*", explain) + + // Test for stale read. + tk.MustExec("set @@tidb_replica_read='closest-replicas';") + rows = tk.MustQuery("explain analyze select * from t as of timestamp(@stale_read_ts_var) where b > 1").Rows() + require.Len(t, rows, 3) + explain = fmt.Sprintf("%v", rows[0]) + require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: (3|4).*", explain) +}