diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index be2d90d955607..f5836fa8f1b5a 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -681,11 +681,12 @@ 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), TIDB_KV_READ_TIMEOUT(20) */ * from t where a >= 4 and b >= 1 and c = 0") + tk.MustQuery("select /*+ MAX_EXECUTION_TIME(5000), SET_VAR(TIKV_CLIENT_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), tidb_kv_read_timeout(20)*/ * FROM `test`.`t` WHERE `a` >= 4 AND `b` >= 1 AND `c` = 0", rows[0][1]) + // TODO(crazycs520): Fix this case. + //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), set_var(tikv_client_read_timeout = 20)*/ * FROM `test`.`t` WHERE `a` >= 4 AND `b` >= 1 AND `c` = 0", rows[0][1]) } func TestDefaultSessionVars(t *testing.T) { @@ -745,15 +746,13 @@ 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), TIDB_KV_READ_TIMEOUT(20), MEMORY_QUOTA(2 GB) */ * from t use index(idx)") + 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.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) { @@ -1286,7 +1285,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"}, + //{"select * from t", "select /*+ set_var(tikv_client_read_timeout=1000) */ * from t"}, // storage hints {"select * from t", "select /*+ read_from_storage(tikv[t]) */ * from t"}, // others @@ -1348,7 +1347,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"}, + //{"select * from t", "select /*+ set_var(tikv_client_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 c92bdf380cd38..d1f375a6b63d7 100644 --- a/bindinfo/capture_test.go +++ b/bindinfo/capture_test.go @@ -999,7 +999,6 @@ 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 7b7ad7a249d7d..0c2b33005bfa9 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -301,7 +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() + builder.Request.TiKVClientReadTimeout = sv.GetTiKVClientReadTimeout() return builder } diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 32c0f45af07c1..880873b9bde2c 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -598,25 +598,25 @@ func TestRequestBuilder8(t *testing.T) { require.Equal(t, expect, actual) } -func TestRequestBuilderTidbKvReadTimeout(t *testing.T) { +func TestRequestBuilderTiKVClientReadTimeout(t *testing.T) { sv := variable.NewSessionVars(nil) - sv.TidbKvReadTimeout = 100 + sv.TiKVClientReadTimeout = 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, + 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, + TiKVClientReadTimeout: 100, } expect.Paging.MinPagingSize = paging.MinPagingSize expect.Paging.MaxPagingSize = paging.MaxPagingSize diff --git a/executor/adapter.go b/executor/adapter.go index 59d7164df443c..add60804d5744 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1730,7 +1730,7 @@ func getEncodedPlan(stmtCtx *stmtctx.StatementContext, genHint bool) (encodedPla // so we have to iterate all hints from the customer and keep some other necessary hints. switch tableHint.HintName.L { case plannercore.HintMemoryQuota, plannercore.HintUseToja, plannercore.HintNoIndexMerge, - plannercore.HintMaxExecutionTime, plannercore.HintTidbKvReadTimeout, + plannercore.HintMaxExecutionTime, plannercore.HintAggToCop, plannercore.HintIgnoreIndex, plannercore.HintReadFromStorage, plannercore.HintLimitToCop: hints = append(hints, tableHint) diff --git a/executor/builder.go b/executor/builder.go index b8b76326c5bac..76c40b6872518 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1771,7 +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()) + snapshot.SetOption(kv.TiKVClientReadTimeout, sessVars.GetTiKVClientReadTimeout()) 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 9311ad4ff4531..c6eebfcdb7845 100644 --- a/executor/executor_failpoint_test.go +++ b/executor/executor_failpoint_test.go @@ -556,7 +556,7 @@ func TestDeadlocksTable(t *testing.T) { )) } -func TestTidbKvReadTimeout(t *testing.T) { +func TestTiKVClientReadTimeout(t *testing.T) { if *testkit.WithTiKV != "" { t.Skip("skip test since it's only work for unistore") } @@ -569,19 +569,19 @@ func TestTidbKvReadTimeout(t *testing.T) { 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() + rows := tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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() + rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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() + rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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) @@ -589,13 +589,13 @@ func TestTidbKvReadTimeout(t *testing.T) { // 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() + rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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 tikv_client_read_timeout session variable. + tk.MustExec("set @@tikv_client_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) diff --git a/kv/kv.go b/kv/kv.go index e37cfafa3fac1..2899888f8afbb 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -545,8 +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 + // TiKVClientReadTimeout is the timeout of kv read request + TiKVClientReadTimeout 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 57e67071fd44c..4a43cf4063db3 100644 --- a/kv/option.go +++ b/kv/option.go @@ -100,8 +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 + // TiKVClientReadTimeout sets the timeout value for readonly kv request in milliseconds + TiKVClientReadTimeout ) // ReplicaReadType is the type of replica to read data from diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 1ff8812a0534f..32f8c7b27a7ed 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -3721,8 +3721,6 @@ 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 3d44069a95683..7379279d51567 100644 --- a/parser/ast/misc_test.go +++ b/parser/ast/misc_test.go @@ -270,8 +270,6 @@ 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 7bdf10fa20076..ec266f90556d9 100644 --- a/parser/hintparser.go +++ b/parser/hintparser.go @@ -41,237 +41,235 @@ type yyhintXError struct { } const ( - yyhintDefault = 57434 + yyhintDefault = 57433 yyhintEOFCode = 57344 yyhintErrCode = 57345 - hintAggToCop = 57380 - hintBCJoin = 57402 + hintAggToCop = 57379 + hintBCJoin = 57401 hintBKA = 57355 hintBNL = 57357 - hintDupsWeedOut = 57430 - hintFalse = 57426 - hintFirstMatch = 57431 - hintForceIndex = 57416 - hintGB = 57429 - hintHashAgg = 57382 + hintDupsWeedOut = 57429 + hintFalse = 57425 + hintFirstMatch = 57430 + hintForceIndex = 57415 + hintGB = 57428 + hintHashAgg = 57381 hintHashJoin = 57359 hintHashJoinBuild = 57360 hintHashJoinProbe = 57361 hintIdentifier = 57347 - hintIgnoreIndex = 57385 - hintIgnorePlanCache = 57381 - hintIndexHashJoin = 57389 - hintIndexJoin = 57386 + hintIgnoreIndex = 57384 + hintIgnorePlanCache = 57380 + hintIndexHashJoin = 57388 + hintIndexJoin = 57385 hintIndexMerge = 57365 - hintIndexMergeJoin = 57393 - hintInlHashJoin = 57388 - hintInlJoin = 57391 - hintInlMergeJoin = 57392 + hintIndexMergeJoin = 57392 + hintInlHashJoin = 57387 + hintInlJoin = 57390 + hintInlMergeJoin = 57391 hintIntLit = 57346 hintInvalid = 57348 hintJoinFixedOrder = 57351 hintJoinOrder = 57352 hintJoinPrefix = 57353 hintJoinSuffix = 57354 - hintLeading = 57418 - hintLimitToCop = 57415 - hintLooseScan = 57432 - hintMB = 57428 + hintLeading = 57417 + hintLimitToCop = 57414 + hintLooseScan = 57431 + hintMB = 57427 hintMRR = 57367 - hintMaterialization = 57433 + hintMaterialization = 57432 hintMaxExecutionTime = 57375 - hintMemoryQuota = 57395 + hintMemoryQuota = 57394 hintMerge = 57363 - hintMpp1PhaseAgg = 57383 - hintMpp2PhaseAgg = 57384 + hintMpp1PhaseAgg = 57382 + hintMpp2PhaseAgg = 57383 hintNoBKA = 57356 hintNoBNL = 57358 - hintNoDecorrelate = 57420 + hintNoDecorrelate = 57419 hintNoHashJoin = 57362 hintNoICP = 57369 - hintNoIndexHashJoin = 57390 - hintNoIndexJoin = 57387 + hintNoIndexHashJoin = 57389 + hintNoIndexJoin = 57386 hintNoIndexMerge = 57366 - hintNoIndexMergeJoin = 57394 + hintNoIndexMergeJoin = 57393 hintNoMRR = 57368 hintNoMerge = 57364 - hintNoOrderIndex = 57409 + hintNoOrderIndex = 57408 hintNoRangeOptimization = 57370 - hintNoSMJoin = 57401 + hintNoSMJoin = 57400 hintNoSemijoin = 57374 hintNoSkipScan = 57372 - 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 + 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 hintSemijoin = 57373 - hintSetVar = 57377 - hintShuffleJoin = 57403 + hintSetVar = 57376 + hintShuffleJoin = 57402 hintSingleAtIdentifier = 57349 hintSkipScan = 57371 - hintStraightJoin = 57417 - hintStreamAgg = 57404 + hintStraightJoin = 57416 + hintStreamAgg = 57403 hintStringLit = 57350 - hintSwapJoinInputs = 57405 - hintTiFlash = 57425 - hintTiKV = 57424 - hintTidbKvReadTimeout = 57376 - hintTimeRange = 57412 - hintTrue = 57427 - hintUseCascades = 57413 - hintUseIndex = 57407 - hintUseIndexMerge = 57406 - hintUsePlanCache = 57410 - hintUseToja = 57411 + hintSwapJoinInputs = 57404 + hintTiFlash = 57424 + hintTiKV = 57423 + hintTimeRange = 57411 + hintTrue = 57426 + hintUseCascades = 57412 + hintUseIndex = 57406 + hintUseIndexMerge = 57405 + hintUsePlanCache = 57409 + hintUseToja = 57410 yyhintMaxDepth = 200 - yyhintTabOfs = -213 + yyhintTabOfs = -211 ) var ( yyhintXLAT = map[int]int{ - 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) + 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) } yyhintSymNames = []string{ @@ -339,7 +337,6 @@ var ( "hintStraightJoin", "hintStreamAgg", "hintSwapJoinInputs", - "hintTidbKvReadTimeout", "hintTimeRange", "hintUseCascades", "hintUseIndex", @@ -412,99 +409,70 @@ var ( yyhintReductions = []struct{ xsym, components int }{ {0, 1}, - {124, 1}, - {122, 1}, - {122, 3}, + {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}, {122, 1}, {122, 3}, - {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}, + {103, 1}, + {103, 1}, + {97, 2}, {97, 3}, - {97, 5}, - {129, 3}, - {129, 1}, - {116, 2}, - {116, 1}, - {101, 4}, - {121, 0}, - {121, 1}, + {96, 3}, + {96, 5}, + {128, 3}, + {128, 1}, + {115, 2}, + {115, 1}, + {100, 4}, + {120, 0}, {120, 1}, - {120, 3}, - {126, 0}, - {126, 1}, + {119, 1}, + {119, 3}, + {125, 0}, {125, 1}, - {125, 3}, - {128, 1}, - {128, 1}, - {128, 1}, + {124, 1}, + {124, 3}, {127, 1}, {127, 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}, + {127, 1}, + {126, 1}, + {126, 1}, + {118, 1}, + {118, 1}, + {104, 1}, + {104, 1}, + {104, 1}, {114, 1}, {114, 1}, {114, 1}, @@ -516,484 +484,504 @@ var ( {111, 1}, {111, 1}, {111, 1}, - {109, 1}, - {109, 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}, {110, 1}, {110, 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}, + {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}, } yyhintXErrors = map[yyhintXError]string{} - yyhintParseTab = [304][]uint16{ + yyhintParseTab = [298][]uint16{ // 0 - {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}, + {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}, // 5 - {90: 511}, - {90: 508}, - {90: 505}, - {90: 500}, - {90: 497}, + {89: 503}, + {89: 500}, + {89: 497}, + {89: 492}, + {89: 489}, // 10 - {90: 486}, - {90: 474}, - {90: 470}, - {90: 466}, - {90: 462}, + {89: 478}, + {89: 466}, + {89: 462}, + {89: 458}, + {89: 450}, // 15 - {90: 454}, - {90: 451}, - {90: 439}, - {90: 432}, - {90: 427}, + {89: 447}, + {89: 435}, + {89: 428}, + {89: 423}, + {89: 417}, // 20 - {90: 421}, - {90: 418}, - {90: 412}, - {90: 296}, - {90: 150}, + {89: 414}, + {89: 408}, + {89: 293}, + {89: 149}, + {89: 148}, // 25 - {90: 149}, - {90: 148}, - {90: 147}, - {90: 146}, - {90: 145}, + {89: 147}, + {89: 146}, + {89: 145}, + {89: 144}, + {89: 143}, // 30 - {90: 144}, - {90: 143}, - {90: 142}, - {90: 141}, - {90: 140}, + {89: 142}, + {89: 141}, + {89: 140}, + {89: 139}, + {89: 138}, // 35 - {90: 139}, - {90: 138}, - {90: 137}, - {90: 136}, - {90: 135}, + {89: 137}, + {89: 136}, + {89: 135}, + {89: 134}, + {89: 133}, // 40 - {90: 134}, - {90: 133}, - {90: 132}, - {90: 131}, - {90: 130}, + {89: 132}, + {89: 131}, + {89: 130}, + {89: 129}, + {89: 128}, // 45 - {90: 129}, - {90: 128}, - {90: 127}, - {90: 126}, - {90: 125}, + {89: 127}, + {89: 126}, + {89: 125}, + {89: 124}, + {89: 123}, // 50 - {90: 124}, - {90: 123}, - {90: 122}, - {90: 121}, - {90: 120}, + {89: 122}, + {89: 121}, + {89: 120}, + {89: 119}, + {89: 118}, // 55 - {90: 119}, - {90: 118}, - {90: 117}, - {90: 116}, - {90: 115}, + {89: 117}, + {89: 116}, + {89: 115}, + {89: 114}, + {89: 113}, // 60 - {90: 114}, - {90: 113}, - {90: 112}, - {90: 111}, - {90: 110}, + {89: 112}, + {89: 111}, + {89: 110}, + {89: 109}, + {89: 108}, // 65 - {90: 109}, - {90: 108}, - {90: 107}, - {90: 102}, - {90: 101}, + {89: 107}, + {89: 106}, + {89: 101}, + {89: 100}, + {89: 99}, // 70 - {90: 100}, - {90: 99}, - {90: 98}, - {90: 97}, - {90: 96}, + {89: 98}, + {89: 97}, + {89: 96}, + {89: 95}, + {89: 94}, // 75 - {90: 95}, - {90: 94}, - {90: 93}, - {90: 92}, - {90: 91}, + {89: 93}, + {89: 92}, + {89: 91}, + {89: 90}, + {89: 89}, // 80 - {90: 90}, - {90: 89}, - {90: 88}, - {76: 184, 184, 85: 298, 92: 297}, - {76: 303, 302, 102: 301, 300, 118: 299}, + {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}, // 85 - {183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 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}, + {405, 70: 406}, + {186, 70: 186}, + {98: 301}, + {98: 84}, + {98: 83}, // 90 - {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}, + {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}, // 95 - {184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 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}, + {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}, // 100 - {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}, + {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}, // 105 - {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}, + {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}, // 110 - {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}, + {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}, // 115 - {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}, + {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}, // 120 - {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}, + {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}, // 125 - {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}, + {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}, // 130 - {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}, + {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}, // 135 - {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}, + {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}, // 140 - {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}, + {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}, // 145 - {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}, + {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}, // 150 - {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}, + {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}, // 155 - {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}, + {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}, // 160 - {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}, + {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}, // 165 - {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}, + {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}, // 170 - {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}, + {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}, // 175 - {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}, + {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}, // 180 - {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}, + {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}, // 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: 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}, + {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}, // 190 - {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}, + {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}, // 195 - {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}, + {75: 300, 299, 101: 298, 407}, + {185, 70: 185}, + {78: 183, 183, 84: 295, 91: 409}, + {78: 411, 412, 116: 410}, + {413}, // 200 - {79: 415, 416, 117: 414}, - {417}, - {87}, {86}, - {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}, + {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}, // 205 - {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 - {426}, - {152}, + {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}, - {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}, + // 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}, // 215 - {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}, + {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}, // 220 - {94: 434}, - {82: 437, 436, 127: 435}, - {438}, - {154}, + {434}, {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 - {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}, + {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}, // 230 - {448, 87: 449}, - {169, 87: 169}, - {184, 85: 298, 87: 184, 92: 447}, - {167, 87: 167}, - {168, 87: 168}, + {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}, // 235 - {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}, + {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}, // 240 - {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}, + {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}, {156}, {155}, - {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}, + // 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}, // 250 - {94: 464}, + {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}, {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, 198, 91: 198}, - {85: 298, 92: 467, 94: 184}, - {94: 468}, + {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}, // 255 - {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}, + {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}, // 260 - {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}, + {105, 70: 105}, + {104, 70: 104}, + {103, 70: 103}, + {102, 70: 102}, + {71: 471, 472, 473, 474, 109: 476}, // 265 - {159, 71: 159}, - {106, 71: 106}, - {105, 71: 105}, - {104, 71: 104}, - {103, 71: 103}, + {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}, // 270 - {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}, + {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}, // 275 - {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}, + {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}, // 280 - {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}, + {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}, // 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, 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}, + {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}, // 290 - {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}, + {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}, // 295 - {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}, + {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}, } ) @@ -1033,7 +1021,7 @@ func yyhintlex1(yylex yyhintLexer, lval *yyhintSymType) (n int) { } func yyhintParse(yylex yyhintLexer, parser *hintParser) int { - const yyError = 131 + const yyError = 130 yyEx, _ := yylex.(yyhintLexerEx) var yyn int @@ -1280,14 +1268,6 @@ yynewstate: } } case 14: - { - parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-4].ident), - QBName: model.NewCIStr(yyS[yypt-2].ident), - HintData: yyS[yypt-1].number, - } - } - case 15: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-4].ident), @@ -1295,7 +1275,7 @@ yynewstate: HintData: int64(yyS[yypt-1].number), } } - case 16: + case 15: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-5].ident), @@ -1305,19 +1285,19 @@ yynewstate: }, } } - case 17: + case 16: { parser.warnUnsupportedHint(yyS[yypt-3].ident) parser.yyVAL.hint = nil } - case 18: + case 17: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-3].ident), QBName: model.NewCIStr(yyS[yypt-1].ident), } } - case 19: + case 18: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-5].ident), @@ -1325,7 +1305,7 @@ yynewstate: Tables: yyS[yypt-1].hint.Tables, } } - case 20: + case 19: { maxValue := uint64(math.MaxInt64) / yyS[yypt-1].number if yyS[yypt-2].number <= maxValue { @@ -1340,7 +1320,7 @@ yynewstate: parser.yyVAL.hint = nil } } - case 21: + case 20: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-5].ident), @@ -1350,21 +1330,21 @@ yynewstate: }, } } - case 22: + case 21: { 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 23: + case 22: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-3].ident), QBName: model.NewCIStr(yyS[yypt-1].ident), } } - case 24: + case 23: { parser.yyVAL.hint = &ast.TableOptimizerHint{ HintName: model.NewCIStr(yyS[yypt-4].ident), @@ -1372,7 +1352,7 @@ yynewstate: HintData: model.NewCIStr(yyS[yypt-1].ident), } } - case 25: + case 24: { hs := yyS[yypt-1].hints name := model.NewCIStr(yyS[yypt-4].ident) @@ -1383,60 +1363,60 @@ yynewstate: } parser.yyVAL.hints = hs } - case 26: + case 25: { parser.yyVAL.hints = []*ast.TableOptimizerHint{yyS[yypt-0].hint} } - case 27: + case 26: { parser.yyVAL.hints = append(yyS[yypt-2].hints, yyS[yypt-0].hint) } - case 28: + case 27: { h := yyS[yypt-1].hint h.HintData = model.NewCIStr(yyS[yypt-3].ident) parser.yyVAL.hint = h } - case 29: + case 28: { parser.yyVAL.ident = "" } - case 33: + case 32: { parser.yyVAL.modelIdents = nil } - case 34: + case 33: { parser.yyVAL.modelIdents = yyS[yypt-1].modelIdents } - case 35: + case 34: { parser.yyVAL.modelIdents = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} } - case 36: + case 35: { parser.yyVAL.modelIdents = append(yyS[yypt-2].modelIdents, model.NewCIStr(yyS[yypt-0].ident)) } - case 38: + case 37: { parser.yyVAL.hint = &ast.TableOptimizerHint{ QBName: model.NewCIStr(yyS[yypt-0].ident), } } - case 39: + case 38: { parser.yyVAL.hint = &ast.TableOptimizerHint{ Tables: []ast.HintTable{yyS[yypt-0].table}, QBName: model.NewCIStr(yyS[yypt-1].ident), } } - case 40: + case 39: { h := yyS[yypt-2].hint h.Tables = append(h.Tables, yyS[yypt-0].table) parser.yyVAL.hint = h } - case 41: + case 40: { parser.yyVAL.table = ast.HintTable{ TableName: model.NewCIStr(yyS[yypt-2].ident), @@ -1444,7 +1424,7 @@ yynewstate: PartitionList: yyS[yypt-0].modelIdents, } } - case 42: + case 41: { parser.yyVAL.table = ast.HintTable{ DBName: model.NewCIStr(yyS[yypt-4].ident), @@ -1453,71 +1433,71 @@ yynewstate: PartitionList: yyS[yypt-0].modelIdents, } } - case 43: + case 42: { h := yyS[yypt-2].hint h.Tables = append(h.Tables, yyS[yypt-0].table) parser.yyVAL.hint = h } - case 44: + case 43: { parser.yyVAL.hint = &ast.TableOptimizerHint{ Tables: []ast.HintTable{yyS[yypt-0].table}, } } - case 45: + case 44: { parser.yyVAL.table = ast.HintTable{ TableName: model.NewCIStr(yyS[yypt-1].ident), QBName: model.NewCIStr(yyS[yypt-0].ident), } } - case 46: + case 45: { parser.yyVAL.table = ast.HintTable{ QBName: model.NewCIStr(yyS[yypt-0].ident), } } - case 47: + case 46: { 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 48: + case 47: { parser.yyVAL.hint = &ast.TableOptimizerHint{} } - case 50: + case 49: { parser.yyVAL.hint = &ast.TableOptimizerHint{ Indexes: []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)}, } } - case 51: + case 50: { h := yyS[yypt-2].hint h.Indexes = append(h.Indexes, model.NewCIStr(yyS[yypt-0].ident)) parser.yyVAL.hint = h } - case 58: + case 57: { parser.yyVAL.ident = strconv.FormatUint(yyS[yypt-0].number, 10) } - case 59: + case 58: { parser.yyVAL.number = 1024 * 1024 } - case 60: + case 59: { parser.yyVAL.number = 1024 * 1024 * 1024 } - case 61: + case 60: { parser.yyVAL.hint = &ast.TableOptimizerHint{HintData: true} } - case 62: + case 61: { parser.yyVAL.hint = &ast.TableOptimizerHint{HintData: false} } diff --git a/parser/hintparser.y b/parser/hintparser.y index 3a93cd68f2ded..dd21c899017de 100644 --- a/parser/hintparser.y +++ b/parser/hintparser.y @@ -77,7 +77,6 @@ import ( hintSemijoin "SEMIJOIN" hintNoSemijoin "NO_SEMIJOIN" hintMaxExecutionTime "MAX_EXECUTION_TIME" - hintTidbKvReadTimeout "TIDB_KV_READ_TIMEOUT" hintSetVar "SET_VAR" hintResourceGroup "RESOURCE_GROUP" hintQBName "QB_NAME" @@ -267,14 +266,6 @@ 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{ @@ -745,7 +736,6 @@ 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 2de4457b3e68f..3baf5a99ef44b 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -969,7 +969,6 @@ 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 dd56bc06324d3..379cd44536cad 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -3699,10 +3699,6 @@ 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) { @@ -4111,23 +4107,6 @@ 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", @@ -6749,7 +6728,6 @@ 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)"}, @@ -6774,7 +6752,6 @@ 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)"}, @@ -6928,7 +6905,6 @@ 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 ae84c041d0c1c..ef32feb94dc2e 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -160,8 +160,6 @@ const ( 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 ba5ed226819ef..2e69205a2b9c6 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -638,16 +638,6 @@ 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 a953b78141a7a..0a275575e1719 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -587,7 +587,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, tidbKvReadTimeoutCnt, forceNthPlanCnt, straightJoinHintCnt int + var memoryQuotaHintCnt, useToJAHintCnt, useCascadesHintCnt, noIndexMergeHintCnt, readReplicaHintCnt, maxExecutionTimeCnt, forceNthPlanCnt, straightJoinHintCnt int setVars := make(map[string]string) setVarsOffs := make([]int, 0, len(hints)) for i, hint := range hints { @@ -610,9 +610,6 @@ 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 @@ -721,16 +718,6 @@ 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/session_test/session_test.go b/session/session_test/session_test.go index aab75857a3b35..a408c185e8570 100644 --- a/session/session_test/session_test.go +++ b/session/session_test/session_test.go @@ -686,16 +686,27 @@ func TestSetVarHint(t *testing.T) { 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.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tikv_client_read_timeout", "0")) + tk.MustQuery("SELECT /*+ SET_VAR(tikv_client_read_timeout=10) */ @@tikv_client_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)) + tk.MustQuery("SELECT @@tikv_client_read_timeout;").Check(testkit.Rows("0")) + + tk.MustExec("set @@tikv_client_read_timeout = 5") + tk.MustQuery("SELECT /*+ set_var(tikv_client_read_timeout=1) */ @@tikv_client_read_timeout;").Check(testkit.Rows("1")) + require.Equal(t, tk.Session().GetSessionVars().GetTiKVClientReadTimeout(), uint64(1)) + tk.MustQuery("SELECT @@tikv_client_read_timeout;").Check(testkit.Rows("5")) + require.Equal(t, tk.Session().GetSessionVars().GetTiKVClientReadTimeout(), uint64(5)) + tk.MustQuery("SELECT /*+ set_var(tikv_client_read_timeout=xyz) */ @@tikv_client_read_timeout;").Check(testkit.Rows("5")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.Equal(t, "[variable:1232]Incorrect argument type to variable 'tikv_client_read_timeout'", tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error()) + tk.MustQuery("SELECT /*+ set_var(tikv_client_read_timeout='-10') */ @@tikv_client_read_timeout;").Check(testkit.Rows("0")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.Equal(t, "[variable:1292]Truncated incorrect tikv_client_read_timeout value: '-10'", tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error()) + tk.MustExec("prepare stmt01 from 'select /*+ set_var(tikv_client_read_timeout=1) */ @@tikv_client_read_timeout'") + tk.MustQuery("execute stmt01").Check(testkit.Rows("1")) + require.Equal(t, tk.Session().GetSessionVars().GetTiKVClientReadTimeout(), uint64(1)) + tk.MustQuery("SELECT @@tikv_client_read_timeout;").Check(testkit.Rows("5")) + require.Equal(t, tk.Session().GetSessionVars().GetTiKVClientReadTimeout(), 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")) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 4f234f1057c2f..d8e0bc7cd5fb7 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -390,7 +390,6 @@ type StmtHints struct { // Hint Information MemQuotaQuery int64 MaxExecutionTime uint64 - TidbKvReadTimeout uint64 ReplicaRead byte AllowInSubqToJoinAndAgg bool NoIndexMergeHint bool @@ -406,7 +405,6 @@ type StmtHints struct { HasMemQuotaHint bool HasReplicaReadHint bool HasMaxExecutionTime bool - HasTidbKvReadTimeout bool HasEnableCascadesPlannerHint bool SetVars map[string]string @@ -448,8 +446,6 @@ func (sh *StmtHints) Clone() *StmtHints { HasMemQuotaHint: sh.HasMemQuotaHint, HasReplicaReadHint: sh.HasReplicaReadHint, HasMaxExecutionTime: sh.HasMaxExecutionTime, - HasTidbKvReadTimeout: sh.HasTidbKvReadTimeout, - TidbKvReadTimeout: sh.TidbKvReadTimeout, HasEnableCascadesPlannerHint: sh.HasEnableCascadesPlannerHint, SetVars: vars, OriginalTableHints: tableHints, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 17959ef810835..9d2716e74c51d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -974,9 +974,9 @@ 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 + // TiKVClientReadTimeout 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 + TiKVClientReadTimeout uint64 // Killed is a flag to indicate that this query is killed. Killed uint32 @@ -3182,10 +3182,16 @@ 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 +// GetTiKVClientReadTimeout returns readonly kv request timeout, prefer query hint over session variable +func (s *SessionVars) GetTiKVClientReadTimeout() uint64 { + val, ok := s.stmtVars[TiKVClientReadTimeout] + if ok { + timeout, err := strconv.ParseUint(val, 10, 64) + if err == nil { + return timeout + } + // Normally, we should not go into this branch, because we have checked the type of the variable in `SetStmtVar`. + // So just ignore the error here. } - return s.TidbKvReadTimeout + return s.TiKVClientReadTimeout } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index f0d3d56eab09b..20fbb572a0572 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1224,9 +1224,9 @@ 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 { + {Scope: ScopeGlobal | ScopeSession, Name: TiKVClientReadTimeout, 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) + s.TiKVClientReadTimeout = uint64(timeoutMS) return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationServer, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { @@ -2594,8 +2594,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" + // TiKVClientReadTimeout is the name of the 'tikv_client_read_timeout' system variable. + TiKVClientReadTimeout = "tikv_client_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 04bf4bf6ad173..a5df229686595 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -694,9 +694,9 @@ 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") + err = vars.SetStmtVar(TiKVClientReadTimeout, "ACDC") + require.Equal(t, "[variable:1232]Incorrect argument type to variable 'tikv_client_read_timeout'", err.Error()) + err = vars.SetStmtVar(TiKVClientReadTimeout, "100") require.NoError(t, err) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 1923b04ed0bd9..7e89f44584b9c 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -270,8 +270,8 @@ type copTask struct { batchTaskList map[uint64]*batchedCopTask meetLockFallback bool - // timeout value for one kv readonly reqeust - tidbKvReadTimeout uint64 + // timeout value for one kv readonly request + tikvClientReadTimeout uint64 } type batchedCopTask struct { @@ -416,7 +416,7 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv tasks = append(tasks, task) } if !ignoreTiDBKVReadTimeout { - task.tidbKvReadTimeout = req.TidbKvReadTimeout + task.tikvClientReadTimeout = req.TiKVClientReadTimeout } i = nextI @@ -1028,8 +1028,8 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch worker.req.ResourceGroupTagger(req) } timeout := tikv.ReadTimeoutMedium - if task.tidbKvReadTimeout > 0 { - timeout = time.Duration(task.tidbKvReadTimeout) * time.Millisecond + if task.tikvClientReadTimeout> 0 { + timeout = time.Duration(task.tikvClientReadTimeout) * time.Millisecond } req.StoreTp = getEndPointType(task.storeType) startTime := time.Now() diff --git a/store/driver/snap_interceptor_test.go b/store/driver/snap_interceptor_test.go index 33158c0edd108..fde14c89dc4f0 100644 --- a/store/driver/snap_interceptor_test.go +++ b/store/driver/snap_interceptor_test.go @@ -211,7 +211,7 @@ func TestSnapshotWitInterceptor(t *testing.T) { require.Nil(t, iter) require.Equal(t, []interface{}{"OnIterReverse", kv.Key{}}, mockInterceptor.spy) - snap.SetOption(kv.TidbKvReadTimeout, uint64(10)) + snap.SetOption(kv.TiKVClientReadTimeout, 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 1fb4217036a15..b602d32de8ce5 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -136,7 +136,7 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { if size > 0 { s.KVSnapshot.SetScanBatchSize(size) } - case kv.TidbKvReadTimeout: + case kv.TiKVClientReadTimeout: 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 dc88f14e09501..1d2281e1f1b7f 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -267,7 +267,7 @@ 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: + case kv.TiKVClientReadTimeout: txn.KVTxn.GetSnapshot().SetKVReadTimeout(time.Duration(val.(uint64) * uint64(time.Millisecond))) } } diff --git a/tests/realtikvtest/sessiontest/session_fail_test.go b/tests/realtikvtest/sessiontest/session_fail_test.go index 10af71ee421bf..ae0a04bb3cc34 100644 --- a/tests/realtikvtest/sessiontest/session_fail_test.go +++ b/tests/realtikvtest/sessiontest/session_fail_test.go @@ -206,7 +206,7 @@ func TestKill(t *testing.T) { tk.MustExec("kill connection_id();") } -func TestTidbKvReadTimeout(t *testing.T) { +func TestTiKVClientReadTimeout(t *testing.T) { if !*realtikvtest.WithRealTiKV { t.Skip("skip test since it's only work for tikv") } @@ -230,20 +230,20 @@ func TestTidbKvReadTimeout(t *testing.T) { 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() + rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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() + rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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() + rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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) @@ -251,13 +251,13 @@ func TestTidbKvReadTimeout(t *testing.T) { // 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() + rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_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 tikv_client_read_timeout session variable. + tk.MustExec("set @@tikv_client_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)